You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by bo...@apache.org on 2014/05/21 18:06:56 UTC

[01/14] STORM-216: Added Authentication and Authorization.

Repository: incubator-storm
Updated Branches:
  refs/heads/security [created] fe5f41aa8


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/backtype/storm/supervisor_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/supervisor_test.clj b/storm-core/test/clj/backtype/storm/supervisor_test.clj
index 1d0804a..3dcd275 100644
--- a/storm-core/test/clj/backtype/storm/supervisor_test.clj
+++ b/storm-core/test/clj/backtype/storm/supervisor_test.clj
@@ -15,12 +15,18 @@
 ;; limitations under the License.
 (ns backtype.storm.supervisor-test
   (:use [clojure test])
+  (:require [conjure.core])
+  (:use [conjure core])
+  (:require [clojure.contrib [string :as contrib-str]])
   (:require [clojure [string :as string]])
   (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter])
+  (:import [backtype.storm.scheduler ISupervisor])
+  (:import [java.util UUID])
   (:use [backtype.storm bootstrap config testing])
   (:use [backtype.storm.daemon common])
   (:require [backtype.storm.daemon [worker :as worker] [supervisor :as supervisor]])
   (:use [conjure core])
+  (:require [clojure.java.io :as io])
   )
 
 (bootstrap)
@@ -252,7 +258,7 @@
                                opts
                                topo-opts
                                ["-Djava.library.path="
-                                (str "-Dlogfile.name=worker-" mock-port ".log")
+                                (str "-Dlogfile.name=" mock-storm-id "-worker-" mock-port ".log")
                                 "-Dstorm.home="
                                 "-Dlogback.configurationFile=/logback/cluster.xml"
                                 (str "-Dstorm.id=" mock-storm-id)
@@ -274,8 +280,10 @@
                                                    topo-string-opts}
                      add-to-classpath mock-cp
                      supervisor-stormdist-root nil
+                     launch-process nil
+                     set-worker-user! nil
                      supervisor/jlp nil
-                     launch-process nil]
+                     supervisor/write-log-metadata! nil]
             (supervisor/launch-worker mock-supervisor
                                       mock-storm-id
                                       mock-port
@@ -293,8 +301,10 @@
                                                    topo-list-opts}
                      add-to-classpath mock-cp
                      supervisor-stormdist-root nil
+                     launch-process nil
+                     set-worker-user! nil
                      supervisor/jlp nil
-                     launch-process nil]
+                     supervisor/write-log-metadata! nil]
             (supervisor/launch-worker mock-supervisor
                                       mock-storm-id
                                       mock-port
@@ -303,6 +313,99 @@
                                                 [0]
                                                 exp-args)))))))
 
+(defn rm-r [f]
+  (if (.isDirectory f)
+    (for [sub (.listFiles f)] (rm-r sub))
+    (.delete f) 
+  ))
+
+(deftest test-worker-launch-command-run-as-user
+  (testing "*.worker.childopts configuration"
+    (let [mock-port "42"
+          mock-storm-id "fake-storm-id"
+          mock-worker-id "fake-worker-id"
+          mock-cp "mock-classpath'quote-on-purpose"
+          storm-local (str "/tmp/" (UUID/randomUUID))
+          worker-script (str storm-local "/workers/" mock-worker-id "/storm-worker-script.sh")
+          exp-launch ["/bin/worker-launcher"
+                      "me"
+                      "worker"
+                      (str storm-local "/workers/" mock-worker-id)
+                      worker-script]
+          exp-script-fn (fn [opts topo-opts]
+                       (str "#!/bin/bash\n'export' 'LD_LIBRARY_PATH=';\n\nexec 'java' '-server'"
+                                " " (shell-cmd opts)
+                                " " (shell-cmd topo-opts)
+                                " '-Djava.library.path='"
+                                " '-Dlogfile.name=" mock-storm-id "-worker-" mock-port ".log'"
+                                " '-Dstorm.home='"
+                                " '-Dlogback.configurationFile=/logback/cluster.xml'"
+                                " '-Dstorm.id=" mock-storm-id "'"
+                                " '-Dworker.id=" mock-worker-id "'"
+                                " '-Dworker.port=" mock-port "'"
+                                " '-cp' 'mock-classpath'\"'\"'quote-on-purpose'"
+                                " 'backtype.storm.daemon.worker'"
+                                " '" mock-storm-id "'"
+                                " '" mock-port "'"
+                                " '" mock-worker-id "';"))]
+      (.mkdirs (io/file storm-local "workers" mock-worker-id))
+      (try
+      (testing "testing *.worker.childopts as strings with extra spaces"
+        (let [string-opts "-Dfoo=bar  -Xmx1024m"
+              topo-string-opts "-Dkau=aux   -Xmx2048m"
+              exp-script (exp-script-fn ["-Dfoo=bar" "-Xmx1024m"]
+                                    ["-Dkau=aux" "-Xmx2048m"])
+              mock-supervisor {:conf {STORM-CLUSTER-MODE :distributed
+                                      STORM-LOCAL-DIR storm-local
+                                      SUPERVISOR-RUN-WORKER-AS-USER true
+                                      WORKER-CHILDOPTS string-opts}}]
+          (stubbing [read-supervisor-storm-conf {TOPOLOGY-WORKER-CHILDOPTS
+                                                 topo-string-opts
+                                                 TOPOLOGY-SUBMITTER-USER "me"}
+                     add-to-classpath mock-cp
+                     supervisor-stormdist-root nil
+                     launch-process nil
+                     set-worker-user! nil
+                     supervisor/java-cmd "java"
+                     supervisor/jlp nil
+                     supervisor/write-log-metadata! nil]
+            (supervisor/launch-worker mock-supervisor
+                                      mock-storm-id
+                                      mock-port
+                                      mock-worker-id)
+            (verify-first-call-args-for-indices launch-process
+                                                [0]
+                                                exp-launch))
+          (is (= (slurp worker-script) exp-script))))
+      (testing "testing *.worker.childopts as list of strings, with spaces in values"
+        (let [list-opts '("-Dopt1='this has a space in it'" "-Xmx1024m")
+              topo-list-opts '("-Dopt2='val with spaces'" "-Xmx2048m")
+              exp-script (exp-script-fn list-opts topo-list-opts)
+              mock-supervisor {:conf {STORM-CLUSTER-MODE :distributed
+                                      STORM-LOCAL-DIR storm-local
+                                      SUPERVISOR-RUN-WORKER-AS-USER true
+                                      WORKER-CHILDOPTS list-opts}}]
+          (stubbing [read-supervisor-storm-conf {TOPOLOGY-WORKER-CHILDOPTS
+                                                 topo-list-opts
+                                                 TOPOLOGY-SUBMITTER-USER "me"}
+                     add-to-classpath mock-cp
+                     supervisor-stormdist-root nil
+                     launch-process nil
+                     set-worker-user! nil
+                     supervisor/java-cmd "java"
+                     supervisor/jlp nil
+                     supervisor/write-log-metadata! nil]
+            (supervisor/launch-worker mock-supervisor
+                                      mock-storm-id
+                                      mock-port
+                                      mock-worker-id)
+            (verify-first-call-args-for-indices launch-process
+                                                [0]
+                                                exp-launch))
+          (is (= (slurp worker-script) exp-script))))
+(finally (rm-r (io/file storm-local)))
+))))
+
 (deftest test-workers-go-bananas
   ;; test that multiple workers are started for a port, and test that
   ;; supervisor shuts down propertly (doesn't shutdown the most
@@ -319,3 +422,142 @@
   ;; TODO just do reassign, and check that cleans up worker states after killing but doesn't get rid of downloaded code
   )
 
+(deftest test-supervisor-data-acls
+  (testing "supervisor-data uses correct ACLs"
+    (let [scheme "digest"
+          digest "storm:thisisapoorpassword"
+          auth-conf {STORM-ZOOKEEPER-AUTH-SCHEME scheme
+                     STORM-ZOOKEEPER-AUTH-PAYLOAD digest}
+          expected-acls supervisor/SUPERVISOR-ZK-ACLS
+          fake-isupervisor (reify ISupervisor
+                             (getSupervisorId [this] nil)
+                             (getAssignmentId [this] nil))]
+      (stubbing [uptime-computer nil
+                 cluster/mk-storm-cluster-state nil
+                 supervisor-state nil
+                 local-hostname nil
+                 mk-timer nil]
+        (supervisor/supervisor-data auth-conf nil fake-isupervisor)
+        (verify-call-times-for cluster/mk-storm-cluster-state 1)
+        (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
+                                            expected-acls)))))
+
+(deftest test-write-log-metadata
+  (testing "supervisor writes correct data to logs metadata file"
+    (let [exp-owner "alice"
+          exp-worker-id "42"
+          exp-storm-id "0123456789"
+          exp-port 4242
+          exp-logs-users ["bob" "charlie" "daryl"]
+          storm-conf {TOPOLOGY-SUBMITTER-USER "alice"
+                      TOPOLOGY-USERS ["charlie" "bob"]
+                      LOGS-USERS ["daryl"]}
+          exp-data {TOPOLOGY-SUBMITTER-USER exp-owner
+                    "worker-id" exp-worker-id
+                    LOGS-USERS exp-logs-users}
+          conf {}]
+      (mocking [supervisor/write-log-metadata-to-yaml-file!]
+        (supervisor/write-log-metadata! storm-conf exp-owner exp-worker-id
+                                        exp-storm-id exp-port conf)
+        (verify-called-once-with-args supervisor/write-log-metadata-to-yaml-file!
+                                      exp-storm-id exp-port exp-data conf)))))
+
+(deftest test-worker-launcher-requires-user
+  (testing "worker-launcher throws on blank user"
+    (mocking [launch-process]
+      (is (thrown-cause-with-msg? java.lang.IllegalArgumentException
+                                  #"(?i).*user cannot be blank.*"
+                                  (supervisor/worker-launcher {} nil ""))))))
+
+(defn found? [sub-str input-str]
+  (if (string? input-str)
+    (contrib-str/substring? sub-str (str input-str))
+    (some? #(contrib-str/substring? sub-str %) input-str)))
+
+(defn not-found? [sub-str input-str]
+    (complement (found? sub-str input-str)))
+
+(deftest test-substitute-childopts-happy-path
+  (testing "worker-launcher replaces ids in childopts"
+    (let [ worker-id "w-01"
+           storm-id "s-01"
+           port 9999
+           childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%STORM-ID%-%WORKER-ID%-%WORKER-PORT%.log"
+           ]
+      (def childopts-with-ids (supervisor/substitute-childopts childopts worker-id storm-id port))
+      (is (not-found? "%WORKER-ID%" childopts-with-ids))
+      (is (found? "w-01" childopts-with-ids))
+      (is (not-found? "%STORM-ID%" childopts-with-ids))
+      (is (found? "s-01" childopts-with-ids))
+      (is (not-found? "%WORKER-PORT%" childopts-with-ids))
+      (is (found? "-9999." childopts-with-ids))
+      (is (not-found? "%ID%" childopts-with-ids))
+      (is (found? "worker-9999" childopts-with-ids) (str childopts-with-ids))
+    )))
+
+(deftest test-substitute-childopts-storm-id-alone
+  (testing "worker-launcher replaces ids in childopts"
+    (let [ worker-id "w-01"
+           storm-id "s-01"
+           port 9999
+           childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%STORM-ID%.log"]
+           (def childopts-with-ids (supervisor/substitute-childopts childopts worker-id storm-id port))
+           (is (not-found? "%WORKER-ID%" childopts-with-ids))
+           (is (not-found? "w-01" childopts-with-ids))
+           (is (not-found? "%STORM-ID%" childopts-with-ids))
+           (is (found? "s-01" childopts-with-ids))
+           (is (not-found? "%WORKER-PORT%" childopts-with-ids))
+           (is (not-found? "-9999." childopts-with-ids))
+           (is (not-found? "%ID%" childopts-with-ids))
+           (is (not-found? "worker-9999" childopts-with-ids) (str childopts-with-ids))     )))
+
+(deftest test-substitute-childopts-no-keys
+  (testing "worker-launcher has no ids to replace in childopts"
+    (let [ worker-id "w-01"
+           storm-id "s-01"
+           port 9999
+           childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log"]
+           (def childopts-with-ids (supervisor/substitute-childopts childopts worker-id storm-id port))
+           (is (not-found? "%WORKER-ID%" childopts-with-ids))
+           (is (not-found? "w-01" childopts-with-ids))
+           (is (not-found? "%STORM-ID%" childopts-with-ids))
+           (is (not-found? "s-01" childopts-with-ids))
+           (is (not-found? "%WORKER-PORT%" childopts-with-ids))
+           (is (not-found? "-9999." childopts-with-ids))
+           (is (not-found? "%ID%" childopts-with-ids))
+           (is (not-found? "worker-9999" childopts-with-ids) (str childopts-with-ids))    )))
+
+(deftest test-substitute-childopts-nil-childopts
+  (testing "worker-launcher has nil childopts"
+    (let [ worker-id "w-01"
+           storm-id "s-01"
+           port 9999
+           childopts nil]
+           (def childopts-with-ids (supervisor/substitute-childopts childopts worker-id storm-id port))
+           (is (not-found? "%WORKER-ID%" childopts-with-ids))
+           (is (not-found? "w-01" childopts-with-ids))
+           (is (not-found? "%STORM-ID%" childopts-with-ids))
+           (is (not-found? "s-01" childopts-with-ids))
+           (is (not-found? "%WORKER-PORT%" childopts-with-ids))
+           (is (not-found? "-9999." childopts-with-ids))
+           (is (not-found? "%ID%" childopts-with-ids))
+           (is (not-found? "worker-9999" childopts-with-ids) (str childopts-with-ids))
+    )))
+
+(deftest test-substitute-childopts-nil-ids
+  (testing "worker-launcher has nil ids"
+    (let [ worker-id nil
+           storm-id "s-01"
+           port 9999
+           childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%STORM-ID%-%WORKER-ID%-%WORKER-PORT%.log"]
+      (def childopts-with-ids (supervisor/substitute-childopts childopts worker-id storm-id port))
+      (is (not-found? "%WORKER-ID%" childopts-with-ids))
+      (is (not-found? "w-01" childopts-with-ids))
+      (is (not-found? "%STORM-ID%" childopts-with-ids))
+      (is (found? "s-01" childopts-with-ids))
+      (is (not-found? "%WORKER-PORT%" childopts-with-ids))
+      (is (found? "-9999." childopts-with-ids))
+      (is (not-found? "%ID%" childopts-with-ids))
+      (is (found? "worker-9999" childopts-with-ids) (str childopts-with-ids))
+      )))
+

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/backtype/storm/transactional_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/transactional_test.clj b/storm-core/test/clj/backtype/storm/transactional_test.clj
index c633aa1..6c9ea44 100644
--- a/storm-core/test/clj/backtype/storm/transactional_test.clj
+++ b/storm-core/test/clj/backtype/storm/transactional_test.clj
@@ -18,10 +18,16 @@
   (:import [backtype.storm.topology TopologyBuilder])
   (:import [backtype.storm.transactional TransactionalSpoutCoordinator ITransactionalSpout ITransactionalSpout$Coordinator TransactionAttempt
             TransactionalTopologyBuilder])
-  (:import [backtype.storm.transactional.state TransactionalState RotatingTransactionalState RotatingTransactionalState$StateInitializer])
+  (:import [backtype.storm.transactional.state TransactionalState TestTransactionalState RotatingTransactionalState RotatingTransactionalState$StateInitializer])
   (:import [backtype.storm.testing CountingBatchBolt MemoryTransactionalSpout
             KeyedCountingBatchBolt KeyedCountingCommitterBolt KeyedSummingBatchBolt
             IdentityBolt CountingCommitBolt OpaqueMemoryTransactionalSpout])
+  (:import [backtype.storm.utils ZookeeperAuthInfo])
+  (:import [org.apache.curator.framework CuratorFramework])
+  (:import [org.apache.curator.framework.api CreateBuilder ProtectACLCreateModePathAndBytesable])
+  (:import [org.apache.zookeeper CreateMode ZooDefs ZooDefs$Ids])
+  (:import [org.mockito Matchers Mockito])
+  (:import [org.mockito.exceptions.base MockitoAssertionError])
   (:use [backtype.storm bootstrap testing])
   (:use [backtype.storm.daemon common])  
   )
@@ -701,4 +707,21 @@
        (verify! {"count" [[2 "apple" 1]
                           [2 "dog" 1]]})
 
-       ))))
\ No newline at end of file
+       ))))
+
+(deftest test-create-node-acl
+  (testing "Creates ZooKeeper nodes with the correct ACLs"
+    (let [curator (Mockito/mock CuratorFramework)
+          builder0 (Mockito/mock CreateBuilder)
+          builder1 (Mockito/mock ProtectACLCreateModePathAndBytesable)
+          expectedAcls ZooDefs$Ids/CREATOR_ALL_ACL]
+      (. (Mockito/when (.create curator)) (thenReturn builder0))
+      (. (Mockito/when (.creatingParentsIfNeeded builder0)) (thenReturn builder1))
+      (. (Mockito/when (.withMode builder1 (Matchers/isA CreateMode))) (thenReturn builder1))
+      (. (Mockito/when (.withACL builder1 (Mockito/anyList))) (thenReturn builder1))
+      (TestTransactionalState/createNode curator "" (byte-array 0) expectedAcls nil)
+      (is (nil?
+        (try
+          (. (Mockito/verify builder1) (withACL expectedAcls))
+        (catch MockitoAssertionError e
+          e)))))))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/backtype/storm/ui_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/ui_test.clj b/storm-core/test/clj/backtype/storm/ui_test.clj
new file mode 100644
index 0000000..3a837c5
--- /dev/null
+++ b/storm-core/test/clj/backtype/storm/ui_test.clj
@@ -0,0 +1,49 @@
+;; 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.ui-test
+  (:use [clojure test])
+  (:use [backtype.storm config])
+  (:use [backtype.storm testing])
+  (:require [backtype.storm.ui [core :as core]])
+  )
+
+(deftest test-authorized-ui-user
+  (testing "allow cluster admin"
+    (let [conf {NIMBUS-ADMINS ["alice"]}]
+      (is (core/authorized-ui-user? "alice" conf {}))))
+
+  (testing "ignore any cluster-set topology.users"
+    (let [conf {TOPOLOGY-USERS ["alice"]}]
+      (is (not (core/authorized-ui-user? "alice" conf {})))))
+
+  (testing "allow cluster ui user"
+    (let [conf {UI-USERS ["alice"]}]
+      (is (core/authorized-ui-user? "alice" conf {}))))
+
+  (testing "allow submitted topology user"
+    (let [topo-conf {TOPOLOGY-USERS ["alice"]}]
+      (is (core/authorized-ui-user? "alice" {} topo-conf))))
+
+  (testing "allow submitted ui user"
+    (let [topo-conf {UI-USERS ["alice"]}]
+      (is (core/authorized-ui-user? "alice" {} topo-conf))))
+
+  (testing "disallow user not in nimbus admin, topo user, or ui user"
+    (is (not (core/authorized-ui-user? "alice" {} {}))))
+
+  (testing "user cannot override nimbus admin"
+    (let [topo-conf {NIMBUS-ADMINS ["alice"]}]
+      (is (not (core/authorized-ui-user? "alice" {} topo-conf))))))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/backtype/storm/utils/ZookeeperServerCnxnFactory_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/utils/ZookeeperServerCnxnFactory_test.clj b/storm-core/test/clj/backtype/storm/utils/ZookeeperServerCnxnFactory_test.clj
new file mode 100644
index 0000000..0b2d1cd
--- /dev/null
+++ b/storm-core/test/clj/backtype/storm/utils/ZookeeperServerCnxnFactory_test.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.utils.ZookeeperServerCnxnFactory-test
+  (:import [backtype.storm.utils ZookeeperServerCnxnFactory])
+  (:use [clojure test])
+)
+
+(deftest test-constructor-throws-runtimeexception-if-port-too-large
+  (is (thrown? RuntimeException (ZookeeperServerCnxnFactory. 65536 42)))
+)
+
+(deftest test-factory
+  (let [zkcf-negative (ZookeeperServerCnxnFactory. -42 42)
+        next-port (+ (.port zkcf-negative) 1)
+        arbitrary-max-clients 42
+        zkcf-next (ZookeeperServerCnxnFactory. next-port arbitrary-max-clients)]
+    ; Test handling negative port
+    (is (not (nil? zkcf-negative)))
+    ; Test max-clients is correctly set.
+    (is (= (-> zkcf-next .factory .getMaxClientCnxnsPerHost) arbitrary-max-clients))
+  )
+)

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/backtype/storm/utils_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/utils_test.clj b/storm-core/test/clj/backtype/storm/utils_test.clj
index 2294893..b2d5958 100644
--- a/storm-core/test/clj/backtype/storm/utils_test.clj
+++ b/storm-core/test/clj/backtype/storm/utils_test.clj
@@ -32,7 +32,7 @@
            Config/STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING expected_ceiling})
         servers ["bogus_server"]
         arbitrary_port 42
-        curator (Utils/newCurator conf servers arbitrary_port)
+        curator (Utils/newCurator conf servers arbitrary_port nil)
         retry (-> curator .getZookeeperClient .getRetryPolicy)
        ]
     (is (.isAssignableFrom ExponentialBackoffRetry (.getClass retry)))
@@ -49,20 +49,68 @@
                        "backtype.storm.security.auth.SimpleTransportPlugin"
                       Config/NIMBUS_HOST ""
                       Config/NIMBUS_THRIFT_PORT 65535
-                     })]
-    (is (thrown? RuntimeException
+                      STORM-NIMBUS-RETRY-TIMES 0})]
+    (is (thrown-cause? RuntimeException
       (NimbusClient/getConfiguredClient storm-conf)))
   )
 )
 
 (deftest test-getConfiguredClient-throws-RunTimeException-on-bad-args
-  (let [storm-conf (read-storm-config)]
-    (is (thrown? TTransportException
+  (let [storm-conf (merge
+                    (read-storm-config)
+                    {STORM-NIMBUS-RETRY-TIMES 0})]
+    (is (thrown-cause? TTransportException
       (NimbusClient. storm-conf "" 65535)
     ))
   )
 )
 
+(deftest test-isZkAuthenticationConfiguredTopology
+    (testing "Returns false on null config"
+      (is (not (Utils/isZkAuthenticationConfiguredTopology nil))))
+    (testing "Returns false on scheme key missing"
+      (is (not (Utils/isZkAuthenticationConfiguredTopology
+          {STORM-ZOOKEEPER-TOPOLOGY-AUTH-SCHEME nil}))))
+    (testing "Returns false on scheme value null"
+      (is (not
+        (Utils/isZkAuthenticationConfiguredTopology
+          {STORM-ZOOKEEPER-TOPOLOGY-AUTH-SCHEME nil}))))
+    (testing "Returns true when scheme set to string"
+      (is
+        (Utils/isZkAuthenticationConfiguredTopology
+          {STORM-ZOOKEEPER-TOPOLOGY-AUTH-SCHEME "foobar"}))))
+
+(deftest test-isZkAuthenticationConfiguredStormServer
+  (let [k "java.security.auth.login.config"
+        oldprop (System/getProperty k)]
+    (try
+      (.remove (System/getProperties) k)
+      (testing "Returns false on null config"
+        (is (not (Utils/isZkAuthenticationConfiguredStormServer nil))))
+      (testing "Returns false on scheme key missing"
+        (is (not (Utils/isZkAuthenticationConfiguredStormServer
+            {STORM-ZOOKEEPER-AUTH-SCHEME nil}))))
+      (testing "Returns false on scheme value null"
+        (is (not
+          (Utils/isZkAuthenticationConfiguredStormServer
+            {STORM-ZOOKEEPER-AUTH-SCHEME nil}))))
+      (testing "Returns true when scheme set to string"
+        (is
+          (Utils/isZkAuthenticationConfiguredStormServer
+            {STORM-ZOOKEEPER-AUTH-SCHEME "foobar"})))
+      (testing "Returns true when java.security.auth.login.config is set"
+        (do
+          (System/setProperty k "anything")
+          (is (Utils/isZkAuthenticationConfiguredStormServer {}))))
+      (testing "Returns false when java.security.auth.login.config is set"
+        (do
+          (System/setProperty k "anything")
+          (is (Utils/isZkAuthenticationConfiguredStormServer {}))))
+    (finally 
+      (if (not-nil? oldprop) 
+        (System/setProperty k oldprop)
+        (.remove (System/getProperties) k))))))
+
 (deftest test-secs-to-millis-long
   (is (= 0 (secs-to-millis-long 0)))
   (is (= 2 (secs-to-millis-long 0.002)))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/storm/trident/state_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/storm/trident/state_test.clj b/storm-core/test/clj/storm/trident/state_test.clj
index 63e38ca..c10c93c 100644
--- a/storm-core/test/clj/storm/trident/state_test.clj
+++ b/storm-core/test/clj/storm/trident/state_test.clj
@@ -19,10 +19,17 @@
   (:import [storm.trident.operation.builtin Count])
   (:import [storm.trident.state OpaqueValue])
   (:import [storm.trident.state CombinerValueUpdater])
+  (:import [storm.trident.topology.state TransactionalState TestTransactionalState])
   (:import [storm.trident.state.map TransactionalMap OpaqueMap])
   (:import [storm.trident.testing MemoryBackingMap MemoryMapState])
+  (:import [backtype.storm.utils ZookeeperAuthInfo])
+  (:import [org.apache.curator.framework CuratorFramework])
+  (:import [org.apache.curator.framework.api CreateBuilder ProtectACLCreateModePathAndBytesable])
+  (:import [org.apache.zookeeper CreateMode ZooDefs ZooDefs$Ids])
+  (:import [org.mockito Matchers Mockito])
+  (:import [org.mockito.exceptions.base MockitoAssertionError])
   (:use [storm.trident testing])
-  (:use [backtype.storm util]))
+  (:use [backtype.storm config util]))
 
 (defn single-remove [map key]
   (-> map (.multiRemove [[key]])))
@@ -103,6 +110,22 @@
     (.commit map 2)
     ))
 
+(deftest test-create-node-acl
+  (testing "Creates ZooKeeper nodes with the correct ACLs"
+    (let [curator (Mockito/mock CuratorFramework)
+          builder0 (Mockito/mock CreateBuilder)
+          builder1 (Mockito/mock ProtectACLCreateModePathAndBytesable)
+          expectedAcls ZooDefs$Ids/CREATOR_ALL_ACL]
+      (. (Mockito/when (.create curator)) (thenReturn builder0))
+      (. (Mockito/when (.creatingParentsIfNeeded builder0)) (thenReturn builder1))
+      (. (Mockito/when (.withMode builder1 (Matchers/isA CreateMode))) (thenReturn builder1))
+      (. (Mockito/when (.withACL builder1 (Mockito/anyList))) (thenReturn builder1))
+      (TestTransactionalState/createNode curator "" (byte-array 0) expectedAcls nil)
+      (is (nil?
+        (try
+          (. (Mockito/verify builder1) (withACL expectedAcls))
+        (catch MockitoAssertionError e
+          e)))))))
 
 (deftest test-memory-map-state-remove
   (let [map (MemoryMapState. (uuid))]


[13/14] STORM-216: Added Authentication and Authorization.

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/clj/backtype/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/executor.clj b/storm-core/src/clj/backtype/storm/daemon/executor.clj
index f133a1b..d193a9f 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
@@ -222,7 +227,7 @@
      :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))
      :type executor-type
@@ -312,7 +317,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
@@ -329,7 +334,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)
 
@@ -341,6 +346,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]
@@ -361,23 +373,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
@@ -393,7 +407,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
@@ -412,7 +426,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))
@@ -424,15 +438,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 task-datas 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
@@ -441,9 +460,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
                                 ))))
@@ -460,7 +479,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 []
@@ -471,8 +490,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)
@@ -506,13 +525,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
@@ -543,7 +563,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
             ))
           
@@ -591,8 +611,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
@@ -616,6 +637,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 task-datas tuple)
                               (let [task-data (get task-datas task-id)
                                     ^IBolt bolt-obj (:object task-data)
@@ -629,6 +655,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)
@@ -639,7 +668,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
 
@@ -673,9 +711,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)
@@ -701,9 +741,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)
@@ -719,8 +762,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)
@@ -743,6 +789,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

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/clj/backtype/storm/daemon/logviewer.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/logviewer.clj b/storm-core/src/clj/backtype/storm/daemon/logviewer.clj
index 2a48f55..36a737f 100644
--- a/storm-core/src/clj/backtype/storm/daemon/logviewer.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/logviewer.clj
@@ -15,38 +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 [org.apache.commons.logging LogFactory])
-  (:import [org.apache.commons.logging.impl Log4JLogger])
   (:import [ch.qos.logback.core FileAppender])
-  (:import [org.apache.log4j Level])
-  (: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.
@@ -59,61 +201,168 @@ Note that if anything goes wrong, this will throw an Error and exit."
       (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 log-level-page [name level]
-  (let [log (LogFactory/getLog name)]
-    (if level
-      (if (instance? Log4JLogger log)
-        (.setLevel (.getLogger log) (Level/toLevel level))))
-    (str "effective log level for " name " is " (.getLevel (.getLogger log)))))
-
-(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 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- 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))))
-  (GET "/loglevel" [:as {cookies :cookies} & m]
-       (log-template (log-level-page (:name m) (:level m))))
+       (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)))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index bf22a1b..0bd5337 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,23 @@
           (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))
+                (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 +916,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 +993,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 +1006,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 +1019,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 +1073,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 +1087,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 +1104,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 +1133,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 +1144,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 +1156,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 +1165,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 +1179,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 +1217,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 +1233,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 +1245,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 +1273,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 +1301,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 +1327,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

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
index 7566a79..1f8e7e1 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,76 @@
 (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 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)
+          (worker-launcher-and-wait conf user ["rmr" (worker-root conf id)] :log-prefix (str "rmr " 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
+        (worker-launcher-and-wait conf user ["rmr" (worker-pid-path conf id pid)] :log-prefix (str "rmr for " 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 +247,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 +303,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 +486,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 +503,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 +536,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 +565,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 +597,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 +651,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)
       ))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/clj/backtype/storm/daemon/task.clj
----------------------------------------------------------------------
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)


[05/14] STORM-216: Added Authentication and Authorization.

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/native/worker-launcher/impl/configuration.h
----------------------------------------------------------------------
diff --git a/storm-core/src/native/worker-launcher/impl/configuration.h b/storm-core/src/native/worker-launcher/impl/configuration.h
new file mode 100644
index 0000000..b0d4814
--- /dev/null
+++ b/storm-core/src/native/worker-launcher/impl/configuration.h
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Ensure that the configuration file and all of the containing directories
+ * are only writable by root. Otherwise, an attacker can change the 
+ * configuration and potentially cause damage.
+ * returns 0 if permissions are ok
+ */
+int check_configuration_permissions(const char* file_name);
+
+// read the given configuration file
+void read_config(const char* config_file);
+
+//method exposed to get the configurations
+char *get_value(const char* key);
+
+//function to return array of values pointing to the key. Values are
+//comma seperated strings.
+char ** get_values(const char* key);
+
+// Extracts array of values from the comma separated list of values.
+char ** extract_values(char *value);
+
+// free the memory returned by get_values
+void free_values(char** values);
+
+//method to free allocated configuration
+void free_configurations();
+

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/native/worker-launcher/impl/main.c
----------------------------------------------------------------------
diff --git a/storm-core/src/native/worker-launcher/impl/main.c b/storm-core/src/native/worker-launcher/impl/main.c
new file mode 100644
index 0000000..7067cf9
--- /dev/null
+++ b/storm-core/src/native/worker-launcher/impl/main.c
@@ -0,0 +1,210 @@
+/**
+ * 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.
+ */
+
+#include "configuration.h"
+#include "worker-launcher.h"
+
+#include <errno.h>
+#include <grp.h>
+#include <limits.h>
+#include <unistd.h>
+#include <signal.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <strings.h>
+#include <sys/stat.h>
+
+#define _STRINGIFY(X) #X
+#define STRINGIFY(X) _STRINGIFY(X)
+#define CONF_FILENAME "worker-launcher.cfg"
+
+#ifndef EXEC_CONF_DIR
+  #error EXEC_CONF_DIR must be defined
+#endif
+
+void display_usage(FILE *stream) {
+  fprintf(stream,
+          "Usage: worker-launcher --checksetup\n");
+  fprintf(stream,
+      "Usage: worker-launcher user command command-args\n");
+  fprintf(stream, "Commands:\n");
+  fprintf(stream, "   initialize stormdist dir: code-dir <code-directory>\n");
+  fprintf(stream, "   remove a file/directory: rmr <directory>\n");
+  fprintf(stream, "   launch a worker: worker <working-directory> <script-to-run>\n");
+  fprintf(stream, "   signal a worker: signal <pid> <signal>\n");
+}
+
+int main(int argc, char **argv) {
+  int invalid_args = 0; 
+  int do_check_setup = 0;
+  
+  LOGFILE = stdout;
+  ERRORFILE = stderr;
+
+  // Minimum number of arguments required to run 
+  // the std. worker-launcher commands is 3
+  // 3 args not needed for checksetup option
+  if (argc < 3) {
+    invalid_args = 1;
+    if (argc == 2) {
+      const char *arg1 = argv[1];
+      if (strcmp("--checksetup", arg1) == 0) {
+        invalid_args = 0;
+        do_check_setup = 1;        
+      }
+    }
+  }
+  
+  if (invalid_args != 0) {
+    display_usage(stdout);
+    return INVALID_ARGUMENT_NUMBER;
+  }
+
+  const char * command = NULL;
+  const char * working_dir = NULL;
+
+  int exit_code = 0;
+
+  char *executable_file = get_executable();
+
+  char *orig_conf_file = STRINGIFY(EXEC_CONF_DIR) "/" CONF_FILENAME;
+  char *conf_file = realpath(orig_conf_file, NULL);
+
+  if (conf_file == NULL) {
+    fprintf(ERRORFILE, "Configuration file %s not found.\n", orig_conf_file);
+    exit(INVALID_CONFIG_FILE);
+  }
+  if (check_configuration_permissions(conf_file) != 0) {
+    exit(INVALID_CONFIG_FILE);
+  }
+  read_config(conf_file);
+  free(conf_file);
+  conf_file = NULL;
+
+  // look up the node manager group in the config file
+  char *nm_group = get_value(LAUNCHER_GROUP_KEY);
+  if (nm_group == NULL) {
+    fprintf(ERRORFILE, "Can't get configured value for %s.\n", LAUNCHER_GROUP_KEY);
+    exit(INVALID_CONFIG_FILE);
+  }
+  struct group *group_info = getgrnam(nm_group);
+  if (group_info == NULL) {
+    fprintf(ERRORFILE, "Can't get group information for %s - %s.\n", nm_group,
+            strerror(errno));
+    fflush(LOGFILE);
+    exit(INVALID_CONFIG_FILE);
+  }
+
+  set_launcher_uid(getuid(), group_info->gr_gid);
+  // if we are running from a setuid executable, make the real uid root
+  setuid(0);
+  // set the real and effective group id to the node manager group
+  setgid(group_info->gr_gid);
+
+  if (check_executor_permissions(executable_file) != 0) {
+    fprintf(ERRORFILE, "Invalid permissions on worker-launcher binary.\n");
+    return INVALID_CONTAINER_EXEC_PERMISSIONS;
+  }
+
+  if (do_check_setup != 0) {
+    // basic setup checks done
+    // verified configs available and valid
+    // verified executor permissions
+    return 0;
+  }
+
+  //checks done for user name
+  if (argv[optind] == NULL) {
+    fprintf(ERRORFILE, "Invalid user name.\n");
+    return INVALID_USER_NAME;
+  }
+
+  int ret = set_user(argv[optind]);
+  if (ret != 0) {
+    return ret;
+  }
+ 
+  optind = optind + 1;
+  command = argv[optind++];
+
+  fprintf(LOGFILE, "main : command provided %s\n",command);
+  fprintf(LOGFILE, "main : user is %s\n", user_detail->pw_name);
+  fflush(LOGFILE);
+
+  if (strcasecmp("code-dir", command) == 0) {
+    if (argc != 4) {
+      fprintf(ERRORFILE, "Incorrect number of arguments (%d vs 4) for code-dir\n",
+	      argc);
+      fflush(ERRORFILE);
+      return INVALID_ARGUMENT_NUMBER;
+    }
+    exit_code = setup_stormdist_dir(argv[optind]);
+  } else if (strcasecmp("rmr", command) == 0) {
+    if (argc != 4) {
+      fprintf(ERRORFILE, "Incorrect number of arguments (%d vs 4) for rmr\n",
+	      argc);
+      fflush(ERRORFILE);
+      return INVALID_ARGUMENT_NUMBER;
+    }
+    exit_code= delete_as_user(user_detail->pw_name, argv[optind],
+                              NULL);
+  } else if (strcasecmp("worker", command) == 0) {
+    if (argc != 5) {
+      fprintf(ERRORFILE, "Incorrect number of arguments (%d vs 5) for worker\n",
+	      argc);
+      fflush(ERRORFILE);
+      return INVALID_ARGUMENT_NUMBER;
+    }
+    working_dir = argv[optind++];
+    exit_code = setup_stormdist_dir(working_dir);
+    if (exit_code == 0) {
+      exit_code = exec_as_user(working_dir, argv[optind]);
+    }
+  } else if (strcasecmp("signal", command) == 0) {
+    if (argc != 5) {
+      fprintf(ERRORFILE, "Incorrect number of arguments (%d vs 5) for signal\n",
+	      argc);
+      fflush(ERRORFILE);
+      return INVALID_ARGUMENT_NUMBER;
+    }
+    char* end_ptr = NULL;
+    char* option = argv[optind++];
+    int container_pid = strtol(option, &end_ptr, 10);
+    if (option == end_ptr || *end_ptr != '\0') {
+      fprintf(ERRORFILE, "Illegal argument for container pid %s\n", option);
+      fflush(ERRORFILE);
+      return INVALID_ARGUMENT_NUMBER;
+    }
+    option = argv[optind++];
+    int signal = strtol(option, &end_ptr, 10);
+    if (option == end_ptr || *end_ptr != '\0') {
+      fprintf(ERRORFILE, "Illegal argument for signal %s\n", option);
+      fflush(ERRORFILE);
+      return INVALID_ARGUMENT_NUMBER;
+    }
+    exit_code = signal_container_as_user(user_detail->pw_name, container_pid, signal);
+  } else {
+    fprintf(ERRORFILE, "Invalid command %s not supported.",command);
+    fflush(ERRORFILE);
+    exit_code = INVALID_COMMAND_PROVIDED;
+  }
+  fclose(LOGFILE);
+  fclose(ERRORFILE);
+  return exit_code;
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/native/worker-launcher/impl/worker-launcher.c
----------------------------------------------------------------------
diff --git a/storm-core/src/native/worker-launcher/impl/worker-launcher.c b/storm-core/src/native/worker-launcher/impl/worker-launcher.c
new file mode 100644
index 0000000..81d7075
--- /dev/null
+++ b/storm-core/src/native/worker-launcher/impl/worker-launcher.c
@@ -0,0 +1,779 @@
+/**
+ * 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.
+ */
+
+#include "configuration.h"
+#include "worker-launcher.h"
+
+#include <dirent.h>
+#include <fcntl.h>
+#include <fts.h>
+#include <errno.h>
+#include <grp.h>
+#include <unistd.h>
+#include <signal.h>
+#include <stdarg.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <sys/stat.h>
+
+static const int DEFAULT_MIN_USERID = 1000;
+
+static const char* DEFAULT_BANNED_USERS[] = {"bin", 0};
+
+//struct to store the user details
+struct passwd *user_detail = NULL;
+
+FILE* LOGFILE = NULL;
+FILE* ERRORFILE = NULL;
+
+static uid_t launcher_uid = -1;
+static gid_t launcher_gid = -1;
+
+char *concatenate(char *concat_pattern, char *return_path_name,
+   int numArgs, ...);
+
+void set_launcher_uid(uid_t user, gid_t group) {
+  launcher_uid = user;
+  launcher_gid = group;
+}
+
+/**
+ * get the executable filename.
+ */
+char* get_executable() {
+  char buffer[PATH_MAX];
+  snprintf(buffer, PATH_MAX, "/proc/%u/exe", getpid());
+  char *filename = malloc(PATH_MAX);
+  if (NULL == filename) {
+    fprintf(ERRORFILE, "malloc failed in get_executable\n");
+    exit(-1);
+  }
+  ssize_t len = readlink(buffer, filename, PATH_MAX);
+  if (len == -1) {
+    fprintf(ERRORFILE, "Can't get executable name from %s - %s\n", buffer,
+            strerror(errno));
+    exit(-1);
+  } else if (len >= PATH_MAX) {
+    fprintf(ERRORFILE, "Executable name %.*s is longer than %d characters.\n",
+            PATH_MAX, filename, PATH_MAX);
+    exit(-1);
+  }
+  filename[len] = '\0';
+  return filename;
+}
+
+int check_executor_permissions(char *executable_file) {
+  errno = 0;
+  char * resolved_path = realpath(executable_file, NULL);
+  if (resolved_path == NULL) {
+    fprintf(ERRORFILE,
+        "Error resolving the canonical name for the executable : %s!",
+        strerror(errno));
+    return -1;
+  }
+
+  struct stat filestat;
+  errno = 0;
+  if (stat(resolved_path, &filestat) != 0) {
+    fprintf(ERRORFILE, 
+            "Could not stat the executable : %s!.\n", strerror(errno));
+    return -1;
+  }
+
+  uid_t binary_euid = filestat.st_uid; // Binary's user owner
+  gid_t binary_gid = filestat.st_gid; // Binary's group owner
+
+  // Effective uid should be root
+  if (binary_euid != 0) {
+    fprintf(LOGFILE,
+        "The worker-launcher binary should be user-owned by root.\n");
+    return -1;
+  }
+
+  if (binary_gid != getgid()) {
+    fprintf(LOGFILE, "The configured nodemanager group %d is different from"
+            " the group of the executable %d\n", getgid(), binary_gid);
+    return -1;
+  }
+
+  // check others do not have read/write/execute permissions
+  if ((filestat.st_mode & S_IROTH) == S_IROTH || (filestat.st_mode & S_IWOTH)
+      == S_IWOTH || (filestat.st_mode & S_IXOTH) == S_IXOTH) {
+    fprintf(LOGFILE,
+            "The worker-launcher binary should not have read or write or"
+            " execute for others.\n");
+    return -1;
+  }
+
+  // Binary should be setuid/setgid executable
+  if ((filestat.st_mode & S_ISUID) == 0) {
+    fprintf(LOGFILE, "The worker-launcher binary should be set setuid.\n");
+    return -1;
+  }
+
+  return 0;
+}
+
+/**
+ * Change the effective user id to limit damage.
+ */
+static int change_effective_user(uid_t user, gid_t group) {
+  if (geteuid() == user && getegid() == group) {
+    return 0;
+  }
+  if (seteuid(0) != 0) {
+    return -1;
+  }
+  if (setegid(group) != 0) {
+    fprintf(LOGFILE, "Failed to set effective group id %d - %s\n", group,
+            strerror(errno));
+    return -1;
+  }
+  if (seteuid(user) != 0) {
+    fprintf(LOGFILE, "Failed to set effective user id %d - %s\n", user,
+            strerror(errno));
+    return -1;
+  }
+  return 0;
+}
+
+/**
+ * Change the real and effective user and group to abandon the super user
+ * priviledges.
+ */
+int change_user(uid_t user, gid_t group) {
+  if (user == getuid() && user == geteuid() && 
+      group == getgid() && group == getegid()) {
+    return 0;
+  }
+
+  if (seteuid(0) != 0) {
+    fprintf(LOGFILE, "unable to reacquire root - %s\n", strerror(errno));
+    fprintf(LOGFILE, "Real: %d:%d; Effective: %d:%d\n",
+	    getuid(), getgid(), geteuid(), getegid());
+    return SETUID_OPER_FAILED;
+  }
+  if (setgid(group) != 0) {
+    fprintf(LOGFILE, "unable to set group to %d - %s\n", group, 
+            strerror(errno));
+    fprintf(LOGFILE, "Real: %d:%d; Effective: %d:%d\n",
+	    getuid(), getgid(), geteuid(), getegid());
+    return SETUID_OPER_FAILED;
+  }
+  if (setuid(user) != 0) {
+    fprintf(LOGFILE, "unable to set user to %d - %s\n", user, strerror(errno));
+    fprintf(LOGFILE, "Real: %d:%d; Effective: %d:%d\n",
+	    getuid(), getgid(), geteuid(), getegid());
+    return SETUID_OPER_FAILED;
+  }
+
+  return 0;
+}
+
+/**
+ * Utility function to concatenate argB to argA using the concat_pattern.
+ */
+char *concatenate(char *concat_pattern, char *return_path_name, 
+                  int numArgs, ...) {
+  va_list ap;
+  va_start(ap, numArgs);
+  int strlen_args = 0;
+  char *arg = NULL;
+  int j;
+  for (j = 0; j < numArgs; j++) {
+    arg = va_arg(ap, char*);
+    if (arg == NULL) {
+      fprintf(LOGFILE, "One of the arguments passed for %s in null.\n",
+          return_path_name);
+      return NULL;
+    }
+    strlen_args += strlen(arg);
+  }
+  va_end(ap);
+
+  char *return_path = NULL;
+  int str_len = strlen(concat_pattern) + strlen_args + 1;
+
+  return_path = (char *) malloc(str_len);
+  if (return_path == NULL) {
+    fprintf(LOGFILE, "Unable to allocate memory for %s.\n", return_path_name);
+    return NULL;
+  }
+  va_start(ap, numArgs);
+  vsnprintf(return_path, str_len, concat_pattern, ap);
+  va_end(ap);
+  return return_path;
+}
+
+char *get_container_launcher_file(const char* work_dir) {
+  return concatenate("%s/%s", "container launcher", 2, work_dir, CONTAINER_SCRIPT);
+}
+
+/**
+ * Get the tmp directory under the working directory
+ */
+char *get_tmp_directory(const char *work_dir) {
+  return concatenate("%s/%s", "tmp dir", 2, work_dir, TMP_DIR);
+}
+
+/**
+ * Load the user information for a given user name.
+ */
+static struct passwd* get_user_info(const char* user) {
+  int string_size = sysconf(_SC_GETPW_R_SIZE_MAX);
+  void* buffer = malloc(string_size + sizeof(struct passwd));
+  if (buffer == NULL) {
+    fprintf(LOGFILE, "Malloc failed in get_user_info\n");
+    return NULL;
+  }
+  struct passwd *result = NULL;
+  if (getpwnam_r(user, buffer, buffer + sizeof(struct passwd), string_size,
+		 &result) != 0) {
+    free(buffer);
+    buffer = NULL;
+    fprintf(LOGFILE, "Can't get user information %s - %s\n", user,
+	    strerror(errno));
+    return NULL;
+  }
+  return result;
+}
+
+/**
+ * Is the user a real user account?
+ * Checks:
+ *   1. Not root
+ *   2. UID is above the minimum configured.
+ *   3. Not in banned user list
+ * Returns NULL on failure
+ */
+struct passwd* check_user(const char *user) {
+  if (strcmp(user, "root") == 0) {
+    fprintf(LOGFILE, "Running as root is not allowed\n");
+    fflush(LOGFILE);
+    return NULL;
+  }
+  char *min_uid_str = get_value(MIN_USERID_KEY);
+  int min_uid = DEFAULT_MIN_USERID;
+  if (min_uid_str != NULL) {
+    char *end_ptr = NULL;
+    min_uid = strtol(min_uid_str, &end_ptr, 10);
+    if (min_uid_str == end_ptr || *end_ptr != '\0') {
+      fprintf(LOGFILE, "Illegal value of %s for %s in configuration\n", 
+	      min_uid_str, MIN_USERID_KEY);
+      fflush(LOGFILE);
+      free(min_uid_str);
+      min_uid_str = NULL;
+      return NULL;
+    }
+    free(min_uid_str);
+    min_uid_str = NULL;
+  }
+  struct passwd *user_info = get_user_info(user);
+  if (NULL == user_info) {
+    fprintf(LOGFILE, "User %s not found\n", user);
+    fflush(LOGFILE);
+    return NULL;
+  }
+  if (user_info->pw_uid < min_uid) {
+    fprintf(LOGFILE, "Requested user %s has id %d, which is below the "
+	    "minimum allowed %d\n", user, user_info->pw_uid, min_uid);
+    fflush(LOGFILE);
+    free(user_info);
+    user_info = NULL;
+    return NULL;
+  }
+  char **banned_users = get_values(BANNED_USERS_KEY);
+  char **banned_user = (banned_users == NULL) ? 
+    (char**) DEFAULT_BANNED_USERS : banned_users;
+  for(; *banned_user; ++banned_user) {
+    if (strcmp(*banned_user, user) == 0) {
+      free(user_info);
+      user_info = NULL;
+      if (banned_users != (char**)DEFAULT_BANNED_USERS) {
+        free_values(banned_users);
+        banned_users = NULL;
+      }
+      fprintf(LOGFILE, "Requested user %s is banned\n", user);
+      return NULL;
+    }
+  }
+  if (banned_users != NULL && banned_users != (char**)DEFAULT_BANNED_USERS) {
+    free_values(banned_users);
+    banned_users = NULL;
+  }
+  return user_info;
+}
+
+/**
+ * function used to populate and user_details structure.
+ */
+int set_user(const char *user) {
+  // free any old user
+  if (user_detail != NULL) {
+    free(user_detail);
+    user_detail = NULL;
+  }
+  user_detail = check_user(user);
+  if (user_detail == NULL) {
+    return -1;
+  }
+
+  if (geteuid() == user_detail->pw_uid) {
+    return 0;
+  }
+
+  if (initgroups(user, user_detail->pw_gid) != 0) {
+    fprintf(LOGFILE, "Error setting supplementary groups for user %s: %s\n",
+        user, strerror(errno));
+    return -1;
+  }
+
+  return change_effective_user(user_detail->pw_uid, user_detail->pw_gid);
+}
+
+/**
+ * Open a file as the node manager and return a file descriptor for it.
+ * Returns -1 on error
+ */
+static int open_file_as_nm(const char* filename) {
+  uid_t user = geteuid();
+  gid_t group = getegid();
+  if (change_effective_user(launcher_uid, launcher_gid) != 0) {
+    return -1;
+  }
+  int result = open(filename, O_RDONLY);
+  if (result == -1) {
+    fprintf(LOGFILE, "Can't open file %s as node manager - %s\n", filename,
+	    strerror(errno));
+  }
+  if (change_effective_user(user, group)) {
+    result = -1;
+  }
+  return result;
+}
+
+/**
+ * Copy a file from a fd to a given filename.
+ * The new file must not exist and it is created with permissions perm.
+ * The input stream is closed.
+ * Return 0 if everything is ok.
+ */
+static int copy_file(int input, const char* in_filename, 
+		     const char* out_filename, mode_t perm) {
+  const int buffer_size = 128*1024;
+  char buffer[buffer_size];
+  int out_fd = open(out_filename, O_WRONLY|O_CREAT|O_EXCL|O_NOFOLLOW, perm);
+  if (out_fd == -1) {
+    fprintf(LOGFILE, "Can't open %s for output - %s\n", out_filename, 
+            strerror(errno));
+    return -1;
+  }
+  ssize_t len = read(input, buffer, buffer_size);
+  while (len > 0) {
+    ssize_t pos = 0;
+    while (pos < len) {
+      ssize_t write_result = write(out_fd, buffer + pos, len - pos);
+      if (write_result <= 0) {
+	fprintf(LOGFILE, "Error writing to %s - %s\n", out_filename,
+		strerror(errno));
+	close(out_fd);
+	return -1;
+      }
+      pos += write_result;
+    }
+    len = read(input, buffer, buffer_size);
+  }
+  if (len < 0) {
+    fprintf(LOGFILE, "Failed to read file %s - %s\n", in_filename, 
+	    strerror(errno));
+    close(out_fd);
+    return -1;
+  }
+  if (close(out_fd) != 0) {
+    fprintf(LOGFILE, "Failed to close file %s - %s\n", out_filename, 
+	    strerror(errno));
+    return -1;
+  }
+  close(input);
+  return 0;
+}
+
+int setup_stormdist(FTSENT* entry, uid_t euser) {
+  if (lchown(entry->fts_path, euser, launcher_gid) != 0) {
+    fprintf(ERRORFILE, "Failure to exec app initialization process - %s\n",
+      strerror(errno));
+     return -1;
+  }
+  mode_t mode = entry->fts_statp->st_mode;
+  mode_t new_mode = (mode & (S_IRWXU)) | S_IRGRP | S_IWGRP;
+  if ((mode & S_IXUSR) == S_IXUSR) {
+    new_mode = new_mode | S_IXGRP;
+  }
+  if ((mode & S_IFDIR) == S_IFDIR) {
+    new_mode = new_mode | S_ISGID;
+  }
+  if (chmod(entry->fts_path, new_mode) != 0) {
+    fprintf(ERRORFILE, "Failure to exec app initialization process - %s\n",
+      strerror(errno));
+    return -1;
+  }
+  return 0;
+}
+
+int setup_stormdist_dir(const char* local_dir) {
+  //This is the same as
+  //> chmod g+rwX -R $local_dir
+  //> chown -no-dereference -R $user:$supervisor-group $local_dir 
+
+  int exit_code = 0;
+  uid_t euser = geteuid();
+
+  if (local_dir == NULL) {
+    fprintf(ERRORFILE, "Path is null\n");
+    exit_code = UNABLE_TO_BUILD_PATH; // may be malloc failed
+  } else {
+    char *(paths[]) = {strndup(local_dir,PATH_MAX), 0};
+    if (paths[0] == NULL) {
+      fprintf(ERRORFILE, "Malloc failed in setup_stormdist_dir\n");
+      return -1;
+    }
+    // check to make sure the directory exists
+    if (access(local_dir, F_OK) != 0) {
+      if (errno == ENOENT) {
+        fprintf(ERRORFILE, "Path does not exist %s\n", local_dir);
+        free(paths[0]);
+        paths[0] = NULL;
+        return UNABLE_TO_BUILD_PATH;
+      }
+    }
+    FTS* tree = fts_open(paths, FTS_PHYSICAL | FTS_XDEV, NULL);
+    FTSENT* entry = NULL;
+    int ret = 0;
+
+    if (tree == NULL) {
+      fprintf(ERRORFILE,
+              "Cannot open file traversal structure for the path %s:%s.\n", 
+              local_dir, strerror(errno));
+      free(paths[0]);
+      paths[0] = NULL;
+      return -1;
+    }
+
+    if (seteuid(0) != 0) {
+      fprintf(ERRORFILE, "Could not become root\n");
+      return -1;
+    }
+
+    while (((entry = fts_read(tree)) != NULL) && exit_code == 0) {
+      switch (entry->fts_info) {
+
+      case FTS_DP:        // A directory being visited in post-order
+      case FTS_DOT:       // A dot directory
+        //NOOP
+        fprintf(LOGFILE, "NOOP: %s\n", entry->fts_path); break;
+      case FTS_D:         // A directory in pre-order
+      case FTS_F:         // A regular file
+      case FTS_SL:        // A symbolic link
+      case FTS_SLNONE:    // A broken symbolic link
+        //TODO it would be good to validate that the file is owned by the correct user first.
+        fprintf(LOGFILE, "visiting: %s\n", entry->fts_path);
+        if (setup_stormdist(entry, euser) != 0) {
+          exit_code = -1;
+        }
+        break;
+      case FTS_DEFAULT:   // Unknown type of file
+      case FTS_DNR:       // Unreadable directory
+      case FTS_NS:        // A file with no stat(2) information
+      case FTS_DC:        // A directory that causes a cycle
+      case FTS_NSOK:      // No stat information requested
+      case FTS_ERR:       // Error return
+      default:
+        fprintf(LOGFILE, "Unexpected...\n"); break;
+        exit_code = -1;
+        break;
+      }
+    }
+    ret = fts_close(tree);
+    free(paths[0]);
+    paths[0] = NULL;
+  }
+  return exit_code;
+}
+
+
+int signal_container_as_user(const char *user, int pid, int sig) {
+  if(pid <= 0) {
+    return INVALID_CONTAINER_PID;
+  }
+
+  if (change_user(user_detail->pw_uid, user_detail->pw_gid) != 0) {
+    return SETUID_OPER_FAILED;
+  }
+
+  //Don't continue if the process-group is not alive anymore.
+  int has_group = 1;
+  if (kill(-pid,0) < 0) {
+    if (kill(pid, 0) < 0) {
+      if (errno == ESRCH) {
+        return INVALID_CONTAINER_PID;
+      }
+      fprintf(LOGFILE, "Error signalling container %d with %d - %s\n",
+	      pid, sig, strerror(errno));
+      return -1;
+    } else {
+      has_group = 0;
+    }
+  }
+
+  if (kill((has_group ? -1 : 1) * pid, sig) < 0) {
+    if(errno != ESRCH) {
+      fprintf(LOGFILE, 
+              "Error signalling process group %d with signal %d - %s\n", 
+              -pid, sig, strerror(errno));
+      fprintf(stderr, 
+              "Error signalling process group %d with signal %d - %s\n", 
+              -pid, sig, strerror(errno));
+      fflush(LOGFILE);
+      return UNABLE_TO_SIGNAL_CONTAINER;
+    } else {
+      return INVALID_CONTAINER_PID;
+    }
+  }
+  fprintf(LOGFILE, "Killing process %s%d with %d\n",
+	  (has_group ? "group " :""), pid, sig);
+  return 0;
+}
+
+/**
+ * Delete a final directory as the node manager user.
+ */
+static int rmdir_as_nm(const char* path) {
+  int user_uid = geteuid();
+  int user_gid = getegid();
+  int ret = change_effective_user(launcher_uid, launcher_gid);
+  if (ret == 0) {
+    if (rmdir(path) != 0) {
+      fprintf(LOGFILE, "rmdir of %s failed - %s\n", path, strerror(errno));
+      ret = -1;
+    }
+  }
+  // always change back
+  if (change_effective_user(user_uid, user_gid) != 0) {
+    ret = -1;
+  }
+  return ret;
+}
+
+/**
+ * Recursively delete the given path.
+ * full_path : the path to delete
+ * needs_tt_user: the top level directory must be deleted by the tt user.
+ */
+static int delete_path(const char *full_path, 
+                       int needs_tt_user) {
+  int exit_code = 0;
+
+  if (full_path == NULL) {
+    fprintf(LOGFILE, "Path is null\n");
+    exit_code = UNABLE_TO_BUILD_PATH; // may be malloc failed
+  } else {
+    char *(paths[]) = {strndup(full_path,PATH_MAX), 0};
+    if (paths[0] == NULL) {
+      fprintf(LOGFILE, "Malloc failed in delete_path\n");
+      return -1;
+    }
+    // check to make sure the directory exists
+    if (access(full_path, F_OK) != 0) {
+      if (errno == ENOENT) {
+        free(paths[0]);
+        paths[0] = NULL;
+        return 0;
+      }
+    }
+    FTS* tree = fts_open(paths, FTS_PHYSICAL | FTS_XDEV, NULL);
+    FTSENT* entry = NULL;
+    int ret = 0;
+
+    if (tree == NULL) {
+      fprintf(LOGFILE,
+              "Cannot open file traversal structure for the path %s:%s.\n", 
+              full_path, strerror(errno));
+      free(paths[0]);
+      paths[0] = NULL;
+      return -1;
+    }
+    while (((entry = fts_read(tree)) != NULL) && exit_code == 0) {
+      switch (entry->fts_info) {
+
+      case FTS_DP:        // A directory being visited in post-order
+        if (!needs_tt_user ||
+            strcmp(entry->fts_path, full_path) != 0) {
+          if (rmdir(entry->fts_accpath) != 0) {
+            fprintf(LOGFILE, "Couldn't delete directory %s - %s\n", 
+                    entry->fts_path, strerror(errno));
+            exit_code = -1;
+          }
+        }
+        break;
+
+      case FTS_F:         // A regular file
+      case FTS_SL:        // A symbolic link
+      case FTS_SLNONE:    // A broken symbolic link
+      case FTS_DEFAULT:   // Unknown type of file
+        if (unlink(entry->fts_accpath) != 0) {
+          fprintf(LOGFILE, "Couldn't delete file %s - %s\n", entry->fts_path,
+                  strerror(errno));
+          exit_code = -1;
+        }
+        break;
+
+      case FTS_DNR:       // Unreadable directory
+        fprintf(LOGFILE, "Unreadable directory %s. Skipping..\n", 
+                entry->fts_path);
+        break;
+
+      case FTS_D:         // A directory in pre-order
+        // if the directory isn't readable, chmod it
+        if ((entry->fts_statp->st_mode & 0200) == 0) {
+          fprintf(LOGFILE, "Unreadable directory %s, chmoding.\n", 
+                  entry->fts_path);
+          if (chmod(entry->fts_accpath, 0700) != 0) {
+            fprintf(LOGFILE, "Error chmoding %s - %s, continuing\n", 
+                    entry->fts_path, strerror(errno));
+          }
+        }
+        break;
+
+      case FTS_NS:        // A file with no stat(2) information
+        // usually a root directory that doesn't exist
+        fprintf(LOGFILE, "Directory not found %s\n", entry->fts_path);
+        break;
+
+      case FTS_DC:        // A directory that causes a cycle
+      case FTS_DOT:       // A dot directory
+      case FTS_NSOK:      // No stat information requested
+        break;
+
+      case FTS_ERR:       // Error return
+        fprintf(LOGFILE, "Error traversing directory %s - %s\n", 
+                entry->fts_path, strerror(entry->fts_errno));
+        exit_code = -1;
+        break;
+        break;
+      default:
+        exit_code = -1;
+        break;
+      }
+    }
+    ret = fts_close(tree);
+    if (exit_code == 0 && ret != 0) {
+      fprintf(LOGFILE, "Error in fts_close while deleting %s\n", full_path);
+      exit_code = -1;
+    }
+    if (needs_tt_user) {
+      // If the delete failed, try a final rmdir as root on the top level.
+      // That handles the case where the top level directory is in a directory
+      // that is owned by the node manager.
+      exit_code = rmdir_as_nm(full_path);
+    }
+    free(paths[0]);
+    paths[0] = NULL;
+  }
+  return exit_code;
+}
+
+int exec_as_user(const char * working_dir, const char * script_file) {
+  char *script_file_dest = NULL;
+  script_file_dest = get_container_launcher_file(working_dir);
+  if (script_file_dest == NULL) {
+    return OUT_OF_MEMORY;
+  }
+
+  // open launch script
+  int script_file_source = open_file_as_nm(script_file);
+  if (script_file_source == -1) {
+    return -1;
+  }
+
+  setsid();
+
+  // give up root privs
+  if (change_user(user_detail->pw_uid, user_detail->pw_gid) != 0) {
+    return SETUID_OPER_FAILED;
+  }
+
+  if (copy_file(script_file_source, script_file, script_file_dest, S_IRWXU) != 0) {
+    return -1;
+  }
+
+  fcloseall();
+  umask(0027);
+  if (chdir(working_dir) != 0) {
+    fprintf(LOGFILE, "Can't change directory to %s -%s\n", working_dir,
+	    strerror(errno));
+    return -1;
+  }
+
+  if (execlp(script_file_dest, script_file_dest, NULL) != 0) {
+    fprintf(LOGFILE, "Couldn't execute the container launch file %s - %s", 
+            script_file_dest, strerror(errno));
+    return UNABLE_TO_EXECUTE_CONTAINER_SCRIPT;
+  }
+ 
+  //Unreachable
+  return -1;
+}
+
+/**
+ * Delete the given directory as the user from each of the directories
+ * user: the user doing the delete
+ * subdir: the subdir to delete (if baseDirs is empty, this is treated as
+           an absolute path)
+ * baseDirs: (optional) the baseDirs where the subdir is located
+ */
+int delete_as_user(const char *user,
+                   const char *subdir,
+                   char* const* baseDirs) {
+  int ret = 0;
+
+  char** ptr;
+
+  // TODO: No switching user? !!!!
+  if (baseDirs == NULL || *baseDirs == NULL) {
+    return delete_path(subdir, 1);
+  }
+  // do the delete
+  for(ptr = (char**)baseDirs; *ptr != NULL; ++ptr) {
+    char* full_path = concatenate("%s/%s", "user subdir", 2,
+                              *ptr, subdir);
+    if (full_path == NULL) {
+      return -1;
+    }
+    int this_ret = delete_path(full_path, strlen(subdir) == 0);
+    free(full_path);
+    full_path = NULL;
+    // delete as much as we can, but remember the error
+    if (this_ret != 0) {
+      ret = this_ret;
+    }
+  }
+  return ret;
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/native/worker-launcher/impl/worker-launcher.h
----------------------------------------------------------------------
diff --git a/storm-core/src/native/worker-launcher/impl/worker-launcher.h b/storm-core/src/native/worker-launcher/impl/worker-launcher.h
new file mode 100644
index 0000000..59ab998
--- /dev/null
+++ b/storm-core/src/native/worker-launcher/impl/worker-launcher.h
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <pwd.h>
+#include <stdio.h>
+#include <sys/types.h>
+
+enum errorcodes {
+  INVALID_ARGUMENT_NUMBER = 1,
+  INVALID_USER_NAME, //2
+  INVALID_COMMAND_PROVIDED, //3
+  // SUPER_USER_NOT_ALLOWED_TO_RUN_TASKS (NOT USED) 4
+  INVALID_NM_ROOT_DIRS = 5,
+  SETUID_OPER_FAILED, //6
+  UNABLE_TO_EXECUTE_CONTAINER_SCRIPT, //7
+  UNABLE_TO_SIGNAL_CONTAINER, //8
+  INVALID_CONTAINER_PID, //9
+  // ERROR_RESOLVING_FILE_PATH (NOT_USED) 10
+  // RELATIVE_PATH_COMPONENTS_IN_FILE_PATH (NOT USED) 11
+  // UNABLE_TO_STAT_FILE (NOT USED) 12
+  // FILE_NOT_OWNED_BY_ROOT (NOT USED) 13
+  // PREPARE_CONTAINER_DIRECTORIES_FAILED (NOT USED) 14
+  // INITIALIZE_CONTAINER_FAILED (NOT USED) 15
+  // PREPARE_CONTAINER_LOGS_FAILED (NOT USED) 16
+  // INVALID_LOG_DIR (NOT USED) 17
+  OUT_OF_MEMORY = 18,
+  // INITIALIZE_DISTCACHEFILE_FAILED (NOT USED) 19
+  INITIALIZE_USER_FAILED = 20,
+  UNABLE_TO_BUILD_PATH, //21
+  INVALID_CONTAINER_EXEC_PERMISSIONS, //22
+  // PREPARE_JOB_LOGS_FAILED (NOT USED) 23
+  INVALID_CONFIG_FILE =  24,
+  SETSID_OPER_FAILED = 25,
+  WRITE_PIDFILE_FAILED = 26
+};
+
+#define LAUNCHER_GROUP_KEY "storm.worker-launcher.group"
+
+#define USER_DIR_PATTERN "%s/usercache/%s"
+#define NM_APP_DIR_PATTERN USER_DIR_PATTERN "/appcache/%s"
+#define CONTAINER_DIR_PATTERN NM_APP_DIR_PATTERN "/%s"
+#define CONTAINER_SCRIPT "launch_container.sh"
+#define CREDENTIALS_FILENAME "container_tokens"
+#define MIN_USERID_KEY "min.user.id"
+#define BANNED_USERS_KEY "banned.users"
+#define TMP_DIR "tmp"
+
+extern struct passwd *user_detail;
+
+// the log file for messages
+extern FILE *LOGFILE;
+// the log file for error messages
+extern FILE *ERRORFILE;
+
+int setup_stormdist_dir(const char* local_dir);
+
+int exec_as_user(const char * working_dir, const char * args);
+
+// delete a directory (or file) recursively as the user. The directory
+// could optionally be relative to the baseDir set of directories (if the same
+// directory appears on multiple disk volumes, the disk volumes should be passed
+// as the baseDirs). If baseDirs is not specified, then dir_to_be_deleted is 
+// assumed as the absolute path
+int delete_as_user(const char *user,
+                   const char *dir_to_be_deleted,
+                   char* const* baseDirs);
+
+// get the executable's filename
+char* get_executable();
+
+/**
+ * Check the permissions on the worker-launcher to make sure that security is
+ * permissible. For this, we need worker-launcher binary to
+ *    * be user-owned by root
+ *    * be group-owned by a configured special group.
+ *    * others do not have any permissions
+ *    * be setuid/setgid
+ * @param executable_file the file to check
+ * @return -1 on error 0 on success.
+ */
+int check_executor_permissions(char *executable_file);
+
+/**
+ * Function used to signal a container launched by the user.
+ * The function sends appropriate signal to the process group
+ * specified by the pid.
+ * @param user the user to send the signal as.
+ * @param pid the process id to send the signal to.
+ * @param sig the signal to send.
+ * @return an errorcode enum value on error, or 0 on success.
+ */
+int signal_container_as_user(const char *user, int pid, int sig);
+
+// set the uid and gid of the launcher.  This is used when doing some
+// priviledged operations for setting the effective uid and gid.
+void set_launcher_uid(uid_t user, gid_t group);
+
+/**
+ * Is the user a real user account?
+ * Checks:
+ *   1. Not root
+ *   2. UID is above the minimum configured.
+ *   3. Not in banned user list
+ * Returns NULL on failure
+ */
+struct passwd* check_user(const char *user);
+
+// set the user
+int set_user(const char *user);
+
+// methods to get the directories
+
+char *get_container_launcher_file(const char* work_dir);
+
+int change_user(uid_t user, gid_t group);

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/native/worker-launcher/test/test-worker-launcher.c
----------------------------------------------------------------------
diff --git a/storm-core/src/native/worker-launcher/test/test-worker-launcher.c b/storm-core/src/native/worker-launcher/test/test-worker-launcher.c
new file mode 100644
index 0000000..412e922
--- /dev/null
+++ b/storm-core/src/native/worker-launcher/test/test-worker-launcher.c
@@ -0,0 +1,340 @@
+/**
+ * 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.
+ */
+#include "configuration.h"
+#include "worker-launcher.h"
+
+#include <errno.h>
+#include <fcntl.h>
+#include <unistd.h>
+#include <signal.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <sys/stat.h>
+#include <sys/wait.h>
+
+#define TEST_ROOT "/tmp/test-worker-launcher"
+#define DONT_TOUCH_FILE "dont-touch-me"
+#define NM_LOCAL_DIRS       TEST_ROOT "/local-1," TEST_ROOT "/local-2," \
+               TEST_ROOT "/local-3," TEST_ROOT "/local-4," TEST_ROOT "/local-5"
+#define NM_LOG_DIRS         TEST_ROOT "/logdir_1," TEST_ROOT "/logdir_2," \
+                            TEST_ROOT "/logdir_3," TEST_ROOT "/logdir_4"
+#define ARRAY_SIZE 1000
+
+static char* username = NULL;
+static char* local_dirs = NULL;
+static char* log_dirs = NULL;
+
+/**
+ * Run the command using the effective user id.
+ * It can't use system, since bash seems to copy the real user id into the
+ * effective id.
+ */
+void run(const char *cmd) {
+  fflush(stdout);
+  fflush(stderr);
+  pid_t child = fork();
+  if (child == -1) {
+    printf("FAIL: failed to fork - %s\n", strerror(errno));
+  } else if (child == 0) {
+    char *cmd_copy = strdup(cmd);
+    char *ptr;
+    int words = 1;
+    for(ptr = strchr(cmd_copy, ' ');  ptr; ptr = strchr(ptr+1, ' ')) {
+      words += 1;
+    }
+    char **argv = malloc(sizeof(char *) * (words + 1));
+    ptr = strtok(cmd_copy, " ");
+    int i = 0;
+    argv[i++] = ptr;
+    while (ptr != NULL) {
+      ptr = strtok(NULL, " ");
+      argv[i++] = ptr;
+    }
+    if (execvp(argv[0], argv) != 0) {
+      printf("FAIL: exec failed in child %s - %s\n", cmd, strerror(errno));
+      exit(42);
+    }
+  } else {
+    int status = 0;
+    if (waitpid(child, &status, 0) <= 0) {
+      printf("FAIL: failed waiting for child process %s pid %d - %s\n", 
+	     cmd, child, strerror(errno));
+      exit(1);
+    }
+    if (!WIFEXITED(status)) {
+      printf("FAIL: process %s pid %d did not exit\n", cmd, child);
+      exit(1);
+    }
+    if (WEXITSTATUS(status) != 0) {
+      printf("FAIL: process %s pid %d exited with error status %d\n", cmd, 
+	     child, WEXITSTATUS(status));
+      exit(1);
+    }
+  }
+}
+
+int write_config_file(char *file_name) {
+  FILE *file;
+  file = fopen(file_name, "w");
+  if (file == NULL) {
+    printf("Failed to open %s.\n", file_name);
+    return EXIT_FAILURE;
+  }
+  fprintf(file, "banned.users=bannedUser\n");
+  fprintf(file, "min.user.id=1000\n");
+  fclose(file);
+  return 0;
+}
+
+void create_nm_roots(char ** nm_roots) {
+  char** nm_root;
+  for(nm_root=nm_roots; *nm_root != NULL; ++nm_root) {
+    if (mkdir(*nm_root, 0755) != 0) {
+      printf("FAIL: Can't create directory %s - %s\n", *nm_root,
+             strerror(errno));
+      exit(1);
+    }
+    char buffer[100000];
+    sprintf(buffer, "%s/usercache", *nm_root);
+    if (mkdir(buffer, 0755) != 0) {
+      printf("FAIL: Can't create directory %s - %s\n", buffer,
+             strerror(errno));
+      exit(1);
+    }
+  }
+}
+
+void test_get_container_launcher_file() {
+  char *expected_file = ("/tmp/launch_container.sh");
+  char *app_dir = "/tmp";
+  char *container_file =  get_container_launcher_file(app_dir);
+  if (strcmp(container_file, expected_file) != 0) {
+    printf("failure to match expected container file %s vs %s\n", container_file,
+           expected_file);
+    exit(1);
+  }
+  free(container_file);
+}
+
+void test_check_user() {
+  printf("\nTesting test_check_user\n");
+  struct passwd *user = check_user(username);
+  if (user == NULL) {
+    printf("FAIL: failed check for user %s\n", username);
+    exit(1);
+  }
+  free(user);
+  if (check_user("lp") != NULL) {
+    printf("FAIL: failed check for system user lp\n");
+    exit(1);
+  }
+  if (check_user("root") != NULL) {
+    printf("FAIL: failed check for system user root\n");
+    exit(1);
+  }
+}
+
+void test_check_configuration_permissions() {
+  printf("\nTesting check_configuration_permissions\n");
+  if (check_configuration_permissions("/etc/passwd") != 0) {
+    printf("FAIL: failed permission check on /etc/passwd\n");
+    exit(1);
+  }
+  if (check_configuration_permissions(TEST_ROOT) == 0) {
+    printf("FAIL: failed permission check on %s\n", TEST_ROOT);
+    exit(1);
+  }
+}
+
+void run_test_in_child(const char* test_name, void (*func)()) {
+  printf("\nRunning test %s in child process\n", test_name);
+  fflush(stdout);
+  fflush(stderr);
+  pid_t child = fork();
+  if (child == -1) {
+    printf("FAIL: fork failed\n");
+    exit(1);
+  } else if (child == 0) {
+    func();
+    exit(0);
+  } else {
+    int status = 0;
+    if (waitpid(child, &status, 0) == -1) {
+      printf("FAIL: waitpid %d failed - %s\n", child, strerror(errno));
+      exit(1);
+    }
+    if (!WIFEXITED(status)) {
+      printf("FAIL: child %d didn't exit - %d\n", child, status);
+      exit(1);
+    }
+    if (WEXITSTATUS(status) != 0) {
+      printf("FAIL: child %d exited with bad status %d\n",
+	     child, WEXITSTATUS(status));
+      exit(1);
+    }
+  }
+}
+
+void test_signal_container() {
+  printf("\nTesting signal_container\n");
+  fflush(stdout);
+  fflush(stderr);
+  pid_t child = fork();
+  if (child == -1) {
+    printf("FAIL: fork failed\n");
+    exit(1);
+  } else if (child == 0) {
+    if (change_user(user_detail->pw_uid, user_detail->pw_gid) != 0) {
+      exit(1);
+    }
+    sleep(3600);
+    exit(0);
+  } else {
+    printf("Child container launched as %d\n", child);
+    if (signal_container_as_user(username, child, SIGQUIT) != 0) {
+      exit(1);
+    }
+    int status = 0;
+    if (waitpid(child, &status, 0) == -1) {
+      printf("FAIL: waitpid failed - %s\n", strerror(errno));
+      exit(1);
+    }
+    if (!WIFSIGNALED(status)) {
+      printf("FAIL: child wasn't signalled - %d\n", status);
+      exit(1);
+    }
+    if (WTERMSIG(status) != SIGQUIT) {
+      printf("FAIL: child was killed with %d instead of %d\n", 
+	     WTERMSIG(status), SIGQUIT);
+      exit(1);
+    }
+  }
+}
+
+void test_signal_container_group() {
+  printf("\nTesting group signal_container\n");
+  fflush(stdout);
+  fflush(stderr);
+  pid_t child = fork();
+  if (child == -1) {
+    printf("FAIL: fork failed\n");
+    exit(1);
+  } else if (child == 0) {
+    setpgrp();
+    if (change_user(user_detail->pw_uid, user_detail->pw_gid) != 0) {
+      exit(1);
+    }
+    sleep(3600);
+    exit(0);
+  }
+  printf("Child container launched as %d\n", child);
+  if (signal_container_as_user(username, child, SIGKILL) != 0) {
+    exit(1);
+  }
+  int status = 0;
+  if (waitpid(child, &status, 0) == -1) {
+    printf("FAIL: waitpid failed - %s\n", strerror(errno));
+    exit(1);
+  }
+  if (!WIFSIGNALED(status)) {
+    printf("FAIL: child wasn't signalled - %d\n", status);
+    exit(1);
+  }
+  if (WTERMSIG(status) != SIGKILL) {
+    printf("FAIL: child was killed with %d instead of %d\n", 
+	   WTERMSIG(status), SIGKILL);
+    exit(1);
+  }
+}
+
+/**
+ * Ensure that the given path and all of the parent directories are created
+ * with the desired permissions.
+ */
+int mkdirs(const char* path, mode_t perm) {
+  char *cmd = malloc(10 + strlen(path));
+  int ret = 0;
+  sprintf(cmd, "mkdir -p %s", path);
+  ret = system(cmd);
+  free(cmd);
+  return ret;
+}
+
+int main(int argc, char **argv) {
+  LOGFILE = stdout;
+  ERRORFILE = stderr;
+  int my_username = 0;
+
+  // clean up any junk from previous run
+  system("chmod -R u=rwx " TEST_ROOT "; rm -fr " TEST_ROOT);
+  
+  if (mkdirs(TEST_ROOT "/logs/userlogs", 0755) != 0) {
+    exit(1);
+  }
+  
+  if (write_config_file(TEST_ROOT "/test.cfg") != 0) {
+    exit(1);
+  }
+  read_config(TEST_ROOT "/test.cfg");
+
+  local_dirs = (char *) malloc (sizeof(char) * ARRAY_SIZE);
+  strcpy(local_dirs, NM_LOCAL_DIRS);
+  log_dirs = (char *) malloc (sizeof(char) * ARRAY_SIZE);
+  strcpy(log_dirs, NM_LOG_DIRS);
+
+  create_nm_roots(extract_values(local_dirs));
+
+  if (getuid() == 0 && argc == 2) {
+    username = argv[1];
+  } else {
+    username = strdup(getpwuid(getuid())->pw_name);
+    my_username = 1;
+  }
+  set_launcher_uid(geteuid(), getegid());
+
+  if (set_user(username)) {
+    exit(1);
+  }
+
+  printf("\nStarting tests\n");
+
+  printf("\nTesting get_container_launcher_file()\n");
+  test_get_container_launcher_file();
+
+  printf("\nTesting check_configuration_permissions()\n");
+  test_check_configuration_permissions();
+
+  printf("\nTesting check_user()\n");
+  test_check_user();
+
+  // the tests that change user need to be run in a subshell, so that
+  // when they change user they don't give up our privs
+  run_test_in_child("test_signal_container", test_signal_container);
+  run_test_in_child("test_signal_container_group", test_signal_container_group);
+
+  seteuid(0);
+  run("rm -fr " TEST_ROOT);
+  printf("\nFinished tests\n");
+
+  if (my_username) {
+    free(username);
+  }
+  free_configurations();
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/py/storm/DistributedRPC.py
----------------------------------------------------------------------
diff --git a/storm-core/src/py/storm/DistributedRPC.py b/storm-core/src/py/storm/DistributedRPC.py
index a7e6ef9..851ad65 100644
--- a/storm-core/src/py/storm/DistributedRPC.py
+++ b/storm-core/src/py/storm/DistributedRPC.py
@@ -64,6 +64,8 @@ class Client(Iface):
       return result.success
     if result.e is not None:
       raise result.e
+    if result.aze is not None:
+      raise result.aze
     raise TApplicationException(TApplicationException.MISSING_RESULT, "execute failed: unknown result");
 
 
@@ -97,6 +99,8 @@ class Processor(Iface, TProcessor):
       result.success = self._handler.execute(args.functionName, args.funcArgs)
     except DRPCExecutionException, e:
       result.e = e
+    except AuthorizationException, aze:
+      result.aze = aze
     oprot.writeMessageBegin("execute", TMessageType.REPLY, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
@@ -185,19 +189,22 @@ class execute_result:
   Attributes:
    - success
    - e
+   - aze
   """
 
   thrift_spec = (
     (0, TType.STRING, 'success', None, None, ), # 0
     (1, TType.STRUCT, 'e', (DRPCExecutionException, DRPCExecutionException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 2
   )
 
   def __hash__(self):
-    return 0 + hash(self.success) + hash(self.e)
+    return 0 + hash(self.success) + hash(self.e) + hash(self.aze)
 
-  def __init__(self, success=None, e=None,):
+  def __init__(self, success=None, e=None, aze=None,):
     self.success = success
     self.e = e
+    self.aze = aze
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -219,6 +226,12 @@ class execute_result:
           self.e.read(iprot)
         else:
           iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -237,6 +250,10 @@ class execute_result:
       oprot.writeFieldBegin('e', TType.STRUCT, 1)
       self.e.write(oprot)
       oprot.writeFieldEnd()
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 2)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/py/storm/DistributedRPCInvocations.py
----------------------------------------------------------------------
diff --git a/storm-core/src/py/storm/DistributedRPCInvocations.py b/storm-core/src/py/storm/DistributedRPCInvocations.py
index 4f951a9..6de2245 100644
--- a/storm-core/src/py/storm/DistributedRPCInvocations.py
+++ b/storm-core/src/py/storm/DistributedRPCInvocations.py
@@ -74,6 +74,8 @@ class Client(Iface):
     result = result_result()
     result.read(self._iprot)
     self._iprot.readMessageEnd()
+    if result.aze is not None:
+      raise result.aze
     return
 
   def fetchRequest(self, functionName):
@@ -104,6 +106,8 @@ class Client(Iface):
     self._iprot.readMessageEnd()
     if result.success is not None:
       return result.success
+    if result.aze is not None:
+      raise result.aze
     raise TApplicationException(TApplicationException.MISSING_RESULT, "fetchRequest failed: unknown result");
 
   def failRequest(self, id):
@@ -132,6 +136,8 @@ class Client(Iface):
     result = failRequest_result()
     result.read(self._iprot)
     self._iprot.readMessageEnd()
+    if result.aze is not None:
+      raise result.aze
     return
 
 
@@ -163,7 +169,10 @@ class Processor(Iface, TProcessor):
     args.read(iprot)
     iprot.readMessageEnd()
     result = result_result()
-    self._handler.result(args.id, args.result)
+    try:
+      self._handler.result(args.id, args.result)
+    except AuthorizationException, aze:
+      result.aze = aze
     oprot.writeMessageBegin("result", TMessageType.REPLY, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
@@ -174,7 +183,10 @@ class Processor(Iface, TProcessor):
     args.read(iprot)
     iprot.readMessageEnd()
     result = fetchRequest_result()
-    result.success = self._handler.fetchRequest(args.functionName)
+    try:
+      result.success = self._handler.fetchRequest(args.functionName)
+    except AuthorizationException, aze:
+      result.aze = aze
     oprot.writeMessageBegin("fetchRequest", TMessageType.REPLY, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
@@ -185,7 +197,10 @@ class Processor(Iface, TProcessor):
     args.read(iprot)
     iprot.readMessageEnd()
     result = failRequest_result()
-    self._handler.failRequest(args.id)
+    try:
+      self._handler.failRequest(args.id)
+    except AuthorizationException, aze:
+      result.aze = aze
     oprot.writeMessageBegin("failRequest", TMessageType.REPLY, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
@@ -270,12 +285,21 @@ class result_args:
     return not (self == other)
 
 class result_result:
+  """
+  Attributes:
+   - aze
+  """
 
   thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1
   )
 
   def __hash__(self):
-    return 0
+    return 0 + hash(self.aze)
+
+  def __init__(self, aze=None,):
+    self.aze = aze
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -286,6 +310,12 @@ class result_result:
       (fname, ftype, fid) = iprot.readFieldBegin()
       if ftype == TType.STOP:
         break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -296,6 +326,10 @@ class result_result:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
     oprot.writeStructBegin('result_result')
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 1)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -381,17 +415,20 @@ class fetchRequest_result:
   """
   Attributes:
    - success
+   - aze
   """
 
   thrift_spec = (
     (0, TType.STRUCT, 'success', (DRPCRequest, DRPCRequest.thrift_spec), None, ), # 0
+    (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1
   )
 
   def __hash__(self):
-    return 0 + hash(self.success)
+    return 0 + hash(self.success) + hash(self.aze)
 
-  def __init__(self, success=None,):
+  def __init__(self, success=None, aze=None,):
     self.success = success
+    self.aze = aze
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -408,6 +445,12 @@ class fetchRequest_result:
           self.success.read(iprot)
         else:
           iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -422,6 +465,10 @@ class fetchRequest_result:
       oprot.writeFieldBegin('success', TType.STRUCT, 0)
       self.success.write(oprot)
       oprot.writeFieldEnd()
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 1)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -504,12 +551,21 @@ class failRequest_args:
     return not (self == other)
 
 class failRequest_result:
+  """
+  Attributes:
+   - aze
+  """
 
   thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1
   )
 
   def __hash__(self):
-    return 0
+    return 0 + hash(self.aze)
+
+  def __init__(self, aze=None,):
+    self.aze = aze
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -520,6 +576,12 @@ class failRequest_result:
       (fname, ftype, fid) = iprot.readFieldBegin()
       if ftype == TType.STOP:
         break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -530,6 +592,10 @@ class failRequest_result:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
     oprot.writeStructBegin('failRequest_result')
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 1)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/py/storm/Nimbus-remote
----------------------------------------------------------------------
diff --git a/storm-core/src/py/storm/Nimbus-remote b/storm-core/src/py/storm/Nimbus-remote
index 4b2ff04..c184fab 100755
--- a/storm-core/src/py/storm/Nimbus-remote
+++ b/storm-core/src/py/storm/Nimbus-remote
@@ -28,6 +28,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print '  void activate(string name)'
   print '  void deactivate(string name)'
   print '  void rebalance(string name, RebalanceOptions options)'
+  print '  void uploadNewCredentials(string name, Credentials creds)'
   print '  string beginFileUpload()'
   print '  void uploadChunk(string location, string chunk)'
   print '  void finishFileUpload(string location)'
@@ -131,6 +132,12 @@ elif cmd == 'rebalance':
     sys.exit(1)
   pp.pprint(client.rebalance(args[0],eval(args[1]),))
 
+elif cmd == 'uploadNewCredentials':
+  if len(args) != 2:
+    print 'uploadNewCredentials requires 2 args'
+    sys.exit(1)
+  pp.pprint(client.uploadNewCredentials(args[0],eval(args[1]),))
+
 elif cmd == 'beginFileUpload':
   if len(args) != 0:
     print 'beginFileUpload requires 0 args'


[08/14] STORM-216: Added Authentication and Authorization.

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/generated/SubmitOptions.java
----------------------------------------------------------------------
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 @@ public class SubmitOptions implements org.apache.thrift.TBase<SubmitOptions, Sub
   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 @@ public class SubmitOptions implements org.apache.thrift.TBase<SubmitOptions, Sub
      * 
      * @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 @@ public class SubmitOptions implements org.apache.thrift.TBase<SubmitOptions, Sub
       switch(fieldId) {
         case 1: // INITIAL_STATUS
           return INITIAL_STATUS;
+        case 2: // CREDS
+          return CREDS;
         default:
           return null;
       }
@@ -114,6 +119,8 @@ public class SubmitOptions implements org.apache.thrift.TBase<SubmitOptions, Sub
     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 @@ public class SubmitOptions implements org.apache.thrift.TBase<SubmitOptions, Sub
     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 @@ public class SubmitOptions implements org.apache.thrift.TBase<SubmitOptions, Sub
   @Override
   public void clear() {
     this.initial_status = null;
+    this.creds = null;
   }
 
   /**
@@ -177,6 +188,29 @@ public class SubmitOptions implements org.apache.thrift.TBase<SubmitOptions, Sub
     }
   }
 
+  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 @@ public class SubmitOptions implements org.apache.thrift.TBase<SubmitOptions, Sub
       }
       break;
 
+    case CREDS:
+      if (value == null) {
+        unset_creds();
+      } else {
+        set_creds((Credentials)value);
+      }
+      break;
+
     }
   }
 
@@ -195,6 +237,9 @@ public class SubmitOptions implements org.apache.thrift.TBase<SubmitOptions, Sub
     case INITIAL_STATUS:
       return get_initial_status();
 
+    case CREDS:
+      return get_creds();
+
     }
     throw new IllegalStateException();
   }
@@ -208,6 +253,8 @@ public class SubmitOptions implements org.apache.thrift.TBase<SubmitOptions, Sub
     switch (field) {
     case INITIAL_STATUS:
       return is_set_initial_status();
+    case CREDS:
+      return is_set_creds();
     }
     throw new IllegalStateException();
   }
@@ -234,6 +281,15 @@ public class SubmitOptions implements org.apache.thrift.TBase<SubmitOptions, Sub
         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 @@ public class SubmitOptions implements org.apache.thrift.TBase<SubmitOptions, Sub
     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 @@ public class SubmitOptions implements org.apache.thrift.TBase<SubmitOptions, Sub
         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 @@ public class SubmitOptions implements org.apache.thrift.TBase<SubmitOptions, Sub
             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 @@ public class SubmitOptions implements org.apache.thrift.TBase<SubmitOptions, Sub
       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 @@ public class SubmitOptions implements org.apache.thrift.TBase<SubmitOptions, Sub
       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();
   }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
----------------------------------------------------------------------
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 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
   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 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
   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 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     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 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
           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 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
             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 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       }
       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 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     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 class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     }
   }
 
+  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 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       }
       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 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     case ERRORS:
       return get_errors();
 
+    case SCHED_STATUS:
+      return get_sched_status();
+
+    case OWNER:
+      return get_owner();
+
     }
     throw new IllegalStateException();
   }
@@ -493,6 +583,10 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       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 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         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 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     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 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         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 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
             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 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       }
       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 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       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();
   }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java
----------------------------------------------------------------------
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 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
   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 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
   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 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
     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 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
           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 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
         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 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
     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 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
     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 class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
     }
   }
 
+  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 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
       }
       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 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
     case STATUS:
       return get_status();
 
+    case SCHED_STATUS:
+      return get_sched_status();
+
+    case OWNER:
+      return get_owner();
+
     }
     throw new IllegalStateException();
   }
@@ -493,6 +583,10 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
       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 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
         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 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
     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 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
         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 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
             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 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
       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 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
       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();
   }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
----------------------------------------------------------------------
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 @@ public class Cluster {
      * 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 @@ public class Cluster {
         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 class Cluster {
     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;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/scheduler/multitenant/DefaultPool.java
----------------------------------------------------------------------
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";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/scheduler/multitenant/FreePool.java
----------------------------------------------------------------------
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();
+      it.remove();
+    }
+    return ret;
+  }
+  
+  @Override
+  public NodeAndSlotCounts getNodeAndSlotCountIfSlotsWereTaken(int slotsNeeded) {
+    int slotsFound = 0;
+    int nodesFound = 0;
+    Iterator<Node> it = _nodes.iterator();
+    while (it.hasNext() && slotsNeeded > 0) {
+      Node n = it.next();
+      nodesFound++;
+      int totalSlots = n.totalSlots();
+      slotsFound += totalSlots;
+      slotsNeeded -= totalSlots;
+    }
+    return new NodeAndSlotCounts(nodesFound, slotsFound);
+  }
+
+  @Override
+  public void scheduleAsNeeded(NodePool... lesserPools) {
+    //No topologies running so NOOP
+  }
+  
+  @Override
+  public String toString() {
+    return "FreePool of "+_nodes.size()+" nodes with "+_totalSlots+" slots";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/scheduler/multitenant/IsolatedPool.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/IsolatedPool.java b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/IsolatedPool.java
new file mode 100644
index 0000000..dc7eded
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/IsolatedPool.java
@@ -0,0 +1,346 @@
+/**
+ * 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.Iterator;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import backtype.storm.Config;
+import backtype.storm.scheduler.SchedulerAssignment;
+import backtype.storm.scheduler.TopologyDetails;
+import backtype.storm.scheduler.WorkerSlot;
+
+/**
+ * A pool of machines that can be used to run isolated topologies
+ */
+public class IsolatedPool extends NodePool {
+  private static final Logger LOG = LoggerFactory.getLogger(IsolatedPool.class);
+  private Map<String, Set<Node>> _topologyIdToNodes = new HashMap<String, Set<Node>>();
+  private HashMap<String, TopologyDetails> _tds = new HashMap<String, TopologyDetails>();
+  private HashSet<String> _isolated = new HashSet<String>();
+  private int _maxNodes;
+  private int _usedNodes;
+
+  public IsolatedPool(int maxNodes) {
+    _maxNodes = maxNodes;
+    _usedNodes = 0;
+  }
+
+  @Override
+  public void addTopology(TopologyDetails td) {
+    String topId = td.getId();
+    LOG.debug("Adding in Topology {}", topId);
+    SchedulerAssignment assignment = _cluster.getAssignmentById(topId);
+    Set<Node> assignedNodes = new HashSet<Node>();
+    if (assignment != null) {
+      for (WorkerSlot ws: assignment.getSlots()) {
+        Node n = _nodeIdToNode.get(ws.getNodeId());
+        assignedNodes.add(n);
+      }
+    }
+    _usedNodes += assignedNodes.size();
+    _topologyIdToNodes.put(topId, assignedNodes);
+    _tds.put(topId, td);
+    if (td.getConf().get(Config.TOPOLOGY_ISOLATED_MACHINES) != null) {
+      _isolated.add(topId);
+    }
+  }
+
+  @Override
+  public boolean canAdd(TopologyDetails td) {
+    //Only add topologies that are not sharing nodes with other topologies
+    String topId = td.getId();
+    SchedulerAssignment assignment = _cluster.getAssignmentById(topId);
+    if (assignment != null) {
+      for (WorkerSlot ws: assignment.getSlots()) {
+        Node n = _nodeIdToNode.get(ws.getNodeId());
+        if (n.getRunningTopologies().size() > 1) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+  
+  @Override
+  public void scheduleAsNeeded(NodePool ... lesserPools) {
+    for (String topId : _topologyIdToNodes.keySet()) {
+      TopologyDetails td = _tds.get(topId);
+      if (_cluster.needsScheduling(td)) {
+        LOG.debug("Scheduling topology {}",topId);
+        Set<Node> allNodes = _topologyIdToNodes.get(topId);
+        Number nodesRequested = (Number) td.getConf().get(Config.TOPOLOGY_ISOLATED_MACHINES);
+        int slotsToUse = 0;
+        if (nodesRequested == null) {
+          slotsToUse = getNodesForNotIsolatedTop(td, allNodes, lesserPools);
+        } else {
+          slotsToUse = getNodesForIsolatedTop(td, allNodes, lesserPools, 
+              nodesRequested.intValue());
+        }
+        //No slots to schedule for some reason, so skip it.
+        if (slotsToUse <= 0) {
+          continue;
+        }
+        
+        RoundRobinSlotScheduler slotSched = 
+          new RoundRobinSlotScheduler(td, slotsToUse, _cluster);
+        
+        LinkedList<Node> sortedNodes = new LinkedList<Node>(allNodes);
+        Collections.sort(sortedNodes, Node.FREE_NODE_COMPARATOR_DEC);
+
+        LOG.debug("Nodes sorted by free space {}", sortedNodes);
+        while (true) {
+          Node n = sortedNodes.remove();
+          if (!slotSched.assignSlotTo(n)) {
+            break;
+          }
+          int freeSlots = n.totalSlotsFree();
+          for (int i = 0; i < sortedNodes.size(); i++) {
+            if (freeSlots >= sortedNodes.get(i).totalSlotsFree()) {
+              sortedNodes.add(i, n);
+              n = null;
+              break;
+            }
+          }
+          if (n != null) {
+            sortedNodes.add(n);
+          }
+        }
+      }
+      Set<Node> found = _topologyIdToNodes.get(topId);
+      int nc = found == null ? 0 : found.size();
+      _cluster.setStatus(topId,"Scheduled Isolated on "+nc+" Nodes");
+    }
+  }
+  
+  /**
+   * Get the nodes needed to schedule an isolated topology.
+   * @param td the topology to be scheduled
+   * @param allNodes the nodes already scheduled for this topology.
+   * This will be updated to include new nodes if needed. 
+   * @param lesserPools node pools we can steal nodes from
+   * @return the number of additional slots that should be used for scheduling.
+   */
+  private int getNodesForIsolatedTop(TopologyDetails td, Set<Node> allNodes,
+      NodePool[] lesserPools, int nodesRequested) {
+    String topId = td.getId();
+    LOG.debug("Topology {} is isolated", topId);
+    int nodesFromUsAvailable = nodesAvailable();
+    int nodesFromOthersAvailable = NodePool.nodesAvailable(lesserPools);
+
+    int nodesUsed = _topologyIdToNodes.get(topId).size();
+    int nodesNeeded = nodesRequested - nodesUsed;
+    LOG.debug("Nodes... requested {} used {} available from us {} " +
+        "avail from other {} needed {}", new Object[] {nodesRequested, 
+        nodesUsed, nodesFromUsAvailable, nodesFromOthersAvailable,
+        nodesNeeded});
+    if ((nodesNeeded - nodesFromUsAvailable) > (_maxNodes - _usedNodes)) {
+      _cluster.setStatus(topId,"Max Nodes("+_maxNodes+") for this user would be exceeded. "
+        + ((nodesNeeded - nodesFromUsAvailable) - (_maxNodes - _usedNodes)) 
+        + " more nodes needed to run topology.");
+      return 0;
+    }
+
+    //In order to avoid going over _maxNodes I may need to steal from
+    // myself even though other pools have free nodes. so figure out how
+    // much each group should provide
+    int nodesNeededFromOthers = Math.min(Math.min(_maxNodes - _usedNodes, 
+        nodesFromOthersAvailable), nodesNeeded);
+    int nodesNeededFromUs = nodesNeeded - nodesNeededFromOthers; 
+    LOG.debug("Nodes... needed from us {} needed from others {}", 
+        nodesNeededFromUs, nodesNeededFromOthers);
+
+    if (nodesNeededFromUs > nodesFromUsAvailable) {
+      _cluster.setStatus(topId, "Not Enough Nodes Available to Schedule Topology");
+      return 0;
+    }
+
+    //Get the nodes
+    Collection<Node> found = NodePool.takeNodes(nodesNeededFromOthers, lesserPools);
+    _usedNodes += found.size();
+    allNodes.addAll(found);
+    Collection<Node> foundMore = takeNodes(nodesNeededFromUs);
+    _usedNodes += foundMore.size();
+    allNodes.addAll(foundMore);
+
+    int totalTasks = td.getExecutors().size();
+    int origRequest = td.getNumWorkers();
+    int slotsRequested = Math.min(totalTasks, origRequest);
+    int slotsUsed = Node.countSlotsUsed(allNodes);
+    int slotsFree = Node.countFreeSlotsAlive(allNodes);
+    int slotsToUse = Math.min(slotsRequested - slotsUsed, slotsFree);
+    if (slotsToUse <= 0) {
+      _cluster.setStatus(topId, "Node has partially crashed, if this situation persists rebalance the topology.");
+    }
+    return slotsToUse;
+  }
+  
+  /**
+   * Get the nodes needed to schedule a non-isolated topology.
+   * @param td the topology to be scheduled
+   * @param allNodes the nodes already scheduled for this topology.
+   * This will be updated to include new nodes if needed. 
+   * @param lesserPools node pools we can steal nodes from
+   * @return the number of additional slots that should be used for scheduling.
+   */
+  private int getNodesForNotIsolatedTop(TopologyDetails td, Set<Node> allNodes,
+      NodePool[] lesserPools) {
+    String topId = td.getId();
+    LOG.debug("Topology {} is not isolated",topId);
+    int totalTasks = td.getExecutors().size();
+    int origRequest = td.getNumWorkers();
+    int slotsRequested = Math.min(totalTasks, origRequest);
+    int slotsUsed = Node.countSlotsUsed(topId, allNodes);
+    int slotsFree = Node.countFreeSlotsAlive(allNodes);
+    //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);
+    LOG.debug("Slots... requested {} used {} free {} available {} to be used {}", 
+        new Object[] {slotsRequested, slotsUsed, slotsFree, slotsAvailable, slotsToUse});
+    if (slotsToUse <= 0) {
+      _cluster.setStatus(topId, "Not Enough Slots Available to Schedule Topology");
+      return 0;
+    }
+    int slotsNeeded = slotsToUse - slotsFree;
+    int numNewNodes = NodePool.getNodeCountIfSlotsWereTaken(slotsNeeded, lesserPools);
+    LOG.debug("Nodes... new {} used {} max {}",
+        new Object[]{numNewNodes, _usedNodes, _maxNodes});
+    if ((numNewNodes + _usedNodes) > _maxNodes) {
+      _cluster.setStatus(topId,"Max Nodes("+_maxNodes+") for this user would be exceeded. " +
+      (numNewNodes - (_maxNodes - _usedNodes)) + " more nodes needed to run topology.");
+      return 0;
+    }
+    
+    Collection<Node> found = NodePool.takeNodesBySlot(slotsNeeded, lesserPools);
+    _usedNodes += found.size();
+    allNodes.addAll(found);
+    return slotsToUse;
+  }
+
+  @Override
+  public Collection<Node> takeNodes(int nodesNeeded) {
+    LOG.debug("Taking {} from {}", nodesNeeded, this);
+    HashSet<Node> ret = new HashSet<Node>();
+    for (Entry<String, Set<Node>> entry: _topologyIdToNodes.entrySet()) {
+      if (!_isolated.contains(entry.getKey())) {
+        Iterator<Node> it = entry.getValue().iterator();
+        while (it.hasNext()) {
+          if (nodesNeeded <= 0) {
+            return ret;
+          }
+          Node n = it.next();
+          it.remove();
+          n.freeAllSlots(_cluster);
+          ret.add(n);
+          nodesNeeded--;
+          _usedNodes--;
+        }
+      }
+    }
+    return ret;
+  }
+  
+  @Override
+  public int nodesAvailable() {
+    int total = 0;
+    for (Entry<String, Set<Node>> entry: _topologyIdToNodes.entrySet()) {
+      if (!_isolated.contains(entry.getKey())) {
+        total += entry.getValue().size();
+      }
+    }
+    return total;
+  }
+  
+  @Override
+  public int slotsAvailable() {
+    int total = 0;
+    for (Entry<String, Set<Node>> entry: _topologyIdToNodes.entrySet()) {
+      if (!_isolated.contains(entry.getKey())) {
+        total += Node.countTotalSlotsAlive(entry.getValue());
+      }
+    }
+    return total;
+  }
+
+  @Override
+  public Collection<Node> takeNodesBySlots(int slotsNeeded) {
+    HashSet<Node> ret = new HashSet<Node>();
+    for (Entry<String, Set<Node>> entry: _topologyIdToNodes.entrySet()) {
+      if (!_isolated.contains(entry.getKey())) {
+        Iterator<Node> it = entry.getValue().iterator();
+        while (it.hasNext()) {
+          Node n = it.next();
+          if (n.isAlive()) {
+            it.remove();
+            _usedNodes--;
+            n.freeAllSlots(_cluster);
+            ret.add(n);
+            slotsNeeded -= n.totalSlots();
+            if (slotsNeeded <= 0) {
+              return ret;
+            }
+          }
+        }
+      }
+    }
+    return ret;
+  }
+  
+  @Override
+  public NodeAndSlotCounts getNodeAndSlotCountIfSlotsWereTaken(int slotsNeeded) {
+    int nodesFound = 0;
+    int slotsFound = 0;
+    for (Entry<String, Set<Node>> entry: _topologyIdToNodes.entrySet()) {
+      if (!_isolated.contains(entry.getKey())) {
+        Iterator<Node> it = entry.getValue().iterator();
+        while (it.hasNext()) {
+          Node n = it.next();
+          if (n.isAlive()) {
+            nodesFound++;
+            int totalSlotsFree = n.totalSlots();
+            slotsFound += totalSlotsFree;
+            slotsNeeded -= totalSlotsFree;
+            if (slotsNeeded <= 0) {
+              return new NodeAndSlotCounts(nodesFound, slotsFound);
+            }
+          }
+        }
+      }
+    }
+    return new NodeAndSlotCounts(nodesFound, slotsFound);
+  }
+  
+  @Override
+  public String toString() {
+    return "IsolatedPool... ";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/scheduler/multitenant/MultitenantScheduler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/MultitenantScheduler.java b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/MultitenantScheduler.java
new file mode 100644
index 0000000..320b388
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/MultitenantScheduler.java
@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package backtype.storm.scheduler.multitenant;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import backtype.storm.Config;
+import backtype.storm.scheduler.Cluster;
+import backtype.storm.scheduler.IScheduler;
+import backtype.storm.scheduler.Topologies;
+import backtype.storm.scheduler.TopologyDetails;
+import backtype.storm.utils.Utils;
+
+public class MultitenantScheduler implements IScheduler {
+  private static final Logger LOG = LoggerFactory.getLogger(MultitenantScheduler.class);
+  @SuppressWarnings("rawtypes")
+  private Map _conf;
+  
+  @Override
+  public void prepare(@SuppressWarnings("rawtypes") Map conf) {
+    _conf = conf;
+  }
+ 
+  private Map<String, Number> getUserConf() {
+    Map<String, Number> ret = (Map<String, Number>)_conf.get(Config.MULTITENANT_SCHEDULER_USER_POOLS);
+    if (ret == null) {
+      ret = new HashMap<String, Number>();
+    } else {
+      ret = new HashMap<String, Number>(ret); 
+    }
+
+    Map fromFile = Utils.findAndReadConfigFile("multitenant-scheduler.yaml", false);
+    Map<String, Number> tmp = (Map<String, Number>)fromFile.get(Config.MULTITENANT_SCHEDULER_USER_POOLS);
+    if (tmp != null) {
+      ret.putAll(tmp);
+    }
+    return ret;
+  }
+
+ 
+  @Override
+  public void schedule(Topologies topologies, Cluster cluster) {
+    LOG.debug("Rerunning scheduling...");
+    Map<String, Node> nodeIdToNode = Node.getAllNodesFrom(cluster);
+    
+    Map<String, Number> userConf = getUserConf();
+    
+    Map<String, IsolatedPool> userPools = new HashMap<String, IsolatedPool>();
+    for (Map.Entry<String, Number> entry : userConf.entrySet()) {
+      userPools.put(entry.getKey(), new IsolatedPool(entry.getValue().intValue()));
+    }
+    DefaultPool defaultPool = new DefaultPool();
+    FreePool freePool = new FreePool();
+    
+    freePool.init(cluster, nodeIdToNode);
+    for (IsolatedPool pool : userPools.values()) {
+      pool.init(cluster, nodeIdToNode);
+    }
+    defaultPool.init(cluster, nodeIdToNode);
+    
+    for (TopologyDetails td: topologies.getTopologies()) {
+      String user = (String)td.getConf().get(Config.TOPOLOGY_SUBMITTER_USER);
+      LOG.debug("Found top {} run by user {}",td.getId(), user);
+      NodePool pool = userPools.get(user);
+      if (pool == null || !pool.canAdd(td)) {
+        pool = defaultPool;
+      }
+      pool.addTopology(td);
+    }
+    
+    //Now schedule all of the topologies that need to be scheduled
+    for (IsolatedPool pool : userPools.values()) {
+      pool.scheduleAsNeeded(freePool, defaultPool);
+    }
+    defaultPool.scheduleAsNeeded(freePool);
+    LOG.debug("Scheduling done...");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/scheduler/multitenant/Node.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/Node.java b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/Node.java
new file mode 100644
index 0000000..2bc2cee
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/Node.java
@@ -0,0 +1,322 @@
+/**
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.Map.Entry;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import backtype.storm.scheduler.Cluster;
+import backtype.storm.scheduler.ExecutorDetails;
+import backtype.storm.scheduler.SchedulerAssignment;
+import backtype.storm.scheduler.SupervisorDetails;
+import backtype.storm.scheduler.WorkerSlot;
+
+/**
+ * Represents a single node in the cluster.
+ */
+public class Node {
+  private static final Logger LOG = LoggerFactory.getLogger(Node.class);
+  private Map<String, Set<WorkerSlot>> _topIdToUsedSlots = new HashMap<String,Set<WorkerSlot>>();
+  private Set<WorkerSlot> _freeSlots = new HashSet<WorkerSlot>();
+  private final String _nodeId;
+  private boolean _isAlive;
+  
+  public Node(String nodeId, Set<Integer> allPorts, boolean isAlive) {
+    _nodeId = nodeId;
+    _isAlive = isAlive;
+    if (_isAlive && allPorts != null) {
+      for (int port: allPorts) {
+        _freeSlots.add(new WorkerSlot(_nodeId, port));
+      }
+    }
+  }
+
+  public String getId() {
+    return _nodeId;
+  }
+  
+  public boolean isAlive() {
+    return _isAlive;
+  }
+  
+  /**
+   * @return a collection of the topology ids currently running on this node
+   */
+  public Collection<String> getRunningTopologies() {
+    return _topIdToUsedSlots.keySet();
+  }
+  
+  public boolean isTotallyFree() {
+    return _topIdToUsedSlots.isEmpty();
+  }
+  
+  public int totalSlotsFree() {
+    return _freeSlots.size();
+  }
+  
+  public int totalSlotsUsed() {
+    int total = 0;
+    for (Set<WorkerSlot> slots: _topIdToUsedSlots.values()) {
+      total += slots.size();
+    }
+    return total;
+  }
+  
+  public int totalSlots() {
+    return totalSlotsFree() + totalSlotsUsed();
+  }
+  
+  public int totalSlotsUsed(String topId) {
+    int total = 0;
+    Set<WorkerSlot> slots = _topIdToUsedSlots.get(topId);
+    if (slots != null) {
+      total = slots.size();
+    }
+    return total;
+  }
+
+  private void validateSlot(WorkerSlot ws) {
+    if (!_nodeId.equals(ws.getNodeId())) {
+      throw new IllegalArgumentException(
+          "Trying to add a slot to the wrong node " + ws + 
+          " is not a part of " + _nodeId);
+    }
+  }
+ 
+  private void addOrphanedSlot(WorkerSlot ws) {
+    if (_isAlive) {
+      throw new IllegalArgumentException("Orphaned Slots " +
+        "only are allowed on dead nodes.");
+    }
+    validateSlot(ws);
+    if (_freeSlots.contains(ws)) {
+      return;
+    }
+    for (Set<WorkerSlot> used: _topIdToUsedSlots.values()) {
+      if (used.contains(ws)) {
+        return;
+      }
+    }
+    _freeSlots.add(ws);
+  }
+ 
+  boolean assignInternal(WorkerSlot ws, String topId, boolean dontThrow) {
+    validateSlot(ws);
+    if (!_freeSlots.remove(ws)) {
+      if (dontThrow) {
+        return true;
+      }
+      throw new IllegalStateException("Assigning a slot that was not free " + ws);
+    }
+    Set<WorkerSlot> usedSlots = _topIdToUsedSlots.get(topId);
+    if (usedSlots == null) {
+      usedSlots = new HashSet<WorkerSlot>();
+      _topIdToUsedSlots.put(topId, usedSlots);
+    }
+    usedSlots.add(ws);
+    return false;
+  }
+  
+  /**
+   * Free all slots on this node.  This will update the Cluster too.
+   * @param cluster the cluster to be updated
+   */
+  public void freeAllSlots(Cluster cluster) {
+    if (!_isAlive) {
+      LOG.warn("Freeing all slots on a dead node {} ",_nodeId);
+    } 
+    for (Entry<String, Set<WorkerSlot>> entry : _topIdToUsedSlots.entrySet()) {
+      cluster.freeSlots(entry.getValue());
+      if (_isAlive) {
+        _freeSlots.addAll(entry.getValue());
+      }
+    }
+    _topIdToUsedSlots = new HashMap<String,Set<WorkerSlot>>();
+  }
+  
+  /**
+   * Frees a single slot in this node
+   * @param ws the slot to free
+   * @param cluster the cluster to update
+   */
+  public void free(WorkerSlot ws, Cluster cluster) {
+    if (_freeSlots.contains(ws)) return;
+    for (Entry<String, Set<WorkerSlot>> entry : _topIdToUsedSlots.entrySet()) {
+      Set<WorkerSlot> slots = entry.getValue();
+      if (slots.remove(ws)) {
+        cluster.freeSlot(ws);
+        if (_isAlive) {
+          _freeSlots.add(ws);
+        }
+        return;
+      }
+    }
+    throw new IllegalArgumentException("Tried to free a slot that was not" +
+    		" part of this node " + _nodeId);
+  }
+   
+  /**
+   * Frees all the slots for a topology.
+   * @param topId the topology to free slots for
+   * @param cluster the cluster to update
+   */
+  public void freeTopology(String topId, Cluster cluster) {
+    Set<WorkerSlot> slots = _topIdToUsedSlots.get(topId);
+    if (slots == null || slots.isEmpty()) return;
+    for (WorkerSlot ws : slots) {
+      cluster.freeSlot(ws);
+      if (_isAlive) {
+        _freeSlots.add(ws);
+      }
+    }
+    _topIdToUsedSlots.remove(topId);
+  }
+ 
+  /**
+   * Assign a free slot on the node to the following topology and executors.
+   * This will update the cluster too.
+   * @param topId the topology to assign a free slot to.
+   * @param executors the executors to run in that slot.
+   * @param cluster the cluster to be updated
+   */
+  public void assign(String topId, Collection<ExecutorDetails> executors, 
+      Cluster cluster) {
+    if (!_isAlive) {
+      throw new IllegalStateException("Trying to adding to a dead node " + _nodeId);
+    }
+    if (_freeSlots.isEmpty()) {
+      throw new IllegalStateException("Trying to assign to a full node " + _nodeId);
+    }
+    if (executors.size() == 0) {
+      LOG.warn("Trying to assign nothing from " + topId + " to " + _nodeId + " (Ignored)");
+    } else {
+      WorkerSlot slot = _freeSlots.iterator().next();
+      cluster.assign(slot, topId, executors);
+      assignInternal(slot, topId, false);
+    }
+  }
+  
+  @Override
+  public boolean equals(Object other) {
+    if (other instanceof Node) {
+      return _nodeId.equals(((Node)other)._nodeId);
+    }
+    return false;
+  }
+  
+  @Override
+  public int hashCode() {
+    return _nodeId.hashCode();
+  }
+  
+  @Override
+  public String toString() {
+    return "Node: " + _nodeId;
+  }
+
+  public static int countSlotsUsed(String topId, Collection<Node> nodes) {
+    int total = 0;
+    for (Node n: nodes) {
+      total += n.totalSlotsUsed(topId);
+    }
+    return total;
+  }
+  
+  public static int countSlotsUsed(Collection<Node> nodes) {
+    int total = 0;
+    for (Node n: nodes) {
+      total += n.totalSlotsUsed();
+    }
+    return total;
+  }
+  
+  public static int countFreeSlotsAlive(Collection<Node> nodes) {
+    int total = 0;
+    for (Node n: nodes) {
+      if (n.isAlive()) {
+        total += n.totalSlotsFree();
+      }
+    }
+    return total;
+  }
+  
+  public static int countTotalSlotsAlive(Collection<Node> nodes) {
+    int total = 0;
+    for (Node n: nodes) {
+      if (n.isAlive()) {
+        total += n.totalSlots();
+      }
+    }
+    return total;
+  }
+  
+  public static Map<String, Node> getAllNodesFrom(Cluster cluster) {
+    Map<String, Node> nodeIdToNode = new HashMap<String, Node>();
+    for (SupervisorDetails sup : cluster.getSupervisors().values()) {
+      //Node ID and supervisor ID are the same.
+      String id = sup.getId();
+      boolean isAlive = !cluster.isBlackListed(id);
+      LOG.debug("Found a {} Node {} {}", 
+          new Object[] {isAlive? "living":"dead", id, sup.getAllPorts()});
+      nodeIdToNode.put(id, new Node(id, sup.getAllPorts(), isAlive));
+    }
+    
+    for (Entry<String, SchedulerAssignment> entry : cluster.getAssignments().entrySet()) {
+      String topId = entry.getValue().getTopologyId();
+      for (WorkerSlot ws: entry.getValue().getSlots()) {
+        String id = ws.getNodeId();
+        Node node = nodeIdToNode.get(id);
+        if (node == null) {
+          LOG.debug("Found an assigned slot on a dead supervisor {}", ws);
+          node = new Node(id, null, false);
+          nodeIdToNode.put(id, node);
+        }
+        if (!node.isAlive()) {
+          //The supervisor on the node down so add an orphaned slot to hold the unsupervised worker 
+          node.addOrphanedSlot(ws);
+        }
+        if (node.assignInternal(ws, topId, true)) {
+          LOG.warn("Bad scheduling state, "+ws+" assigned multiple workers, unassigning everything...");
+          node.free(ws, cluster);
+        }
+      }
+    }
+    
+    return nodeIdToNode;
+  }
+  
+  /**
+   * Used to sort a list of nodes so the node with the most free slots comes
+   * first.
+   */
+  public static final Comparator<Node> FREE_NODE_COMPARATOR_DEC = new Comparator<Node>() {
+    @Override
+    public int compare(Node o1, Node o2) {
+      return o2.totalSlotsFree() - o1.totalSlotsFree();
+    }
+  };
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/scheduler/multitenant/NodePool.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/NodePool.java b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/NodePool.java
new file mode 100644
index 0000000..21d1577
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/NodePool.java
@@ -0,0 +1,296 @@
+/**
+ * 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.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import backtype.storm.Config;
+import backtype.storm.scheduler.Cluster;
+import backtype.storm.scheduler.ExecutorDetails;
+import backtype.storm.scheduler.SchedulerAssignment;
+import backtype.storm.scheduler.TopologyDetails;
+import backtype.storm.scheduler.WorkerSlot;
+
+/**
+ * A pool of nodes that can be used to run topologies.
+ */
+public abstract class NodePool {
+  protected Cluster _cluster;
+  protected Map<String, Node> _nodeIdToNode;
+  
+  public static class NodeAndSlotCounts {
+    public final int _nodes;
+    public final int _slots;
+    
+    public NodeAndSlotCounts(int nodes, int slots) {
+      _nodes = nodes;
+      _slots = slots;
+    }
+  }
+
+  /**
+   * Place executors into slots in a round robin way, taking into account
+   * component spreading among different hosts.
+   */
+  public static class RoundRobinSlotScheduler {
+    private Map<String,Set<String>> _nodeToComps;
+    private HashMap<String, List<ExecutorDetails>> _spreadToSchedule;
+    private LinkedList<Set<ExecutorDetails>> _slots;
+    private Set<ExecutorDetails> _lastSlot;
+    private Cluster _cluster;
+    private String _topId;
+    
+    /**
+     * Create a new scheduler for a given topology
+     * @param td the topology to schedule
+     * @param slotsToUse the number of slots to use for the executors left to 
+     * schedule.
+     * @param cluster the cluster to schedule this on. 
+     */
+    public RoundRobinSlotScheduler(TopologyDetails td, int slotsToUse, 
+        Cluster cluster) {
+      _topId = td.getId();
+      _cluster = cluster;
+      
+      Map<ExecutorDetails, String> execToComp = td.getExecutorToComponent();
+      SchedulerAssignment assignment = _cluster.getAssignmentById(_topId);
+      _nodeToComps = new HashMap<String, Set<String>>();
+
+      if (assignment != null) {
+        Map<ExecutorDetails, WorkerSlot> execToSlot = assignment.getExecutorToSlot();
+        
+        for (Entry<ExecutorDetails, WorkerSlot> entry: execToSlot.entrySet()) {
+          String nodeId = entry.getValue().getNodeId();
+          Set<String> comps = _nodeToComps.get(nodeId);
+          if (comps == null) {
+            comps = new HashSet<String>();
+            _nodeToComps.put(nodeId, comps);
+          }
+          comps.add(execToComp.get(entry.getKey()));
+        }
+      }
+      
+      _spreadToSchedule = new HashMap<String, List<ExecutorDetails>>();
+      List<String> spreadComps = (List<String>)td.getConf().get(Config.TOPOLOGY_SPREAD_COMPONENTS);
+      if (spreadComps != null) {
+        for (String comp: spreadComps) {
+          _spreadToSchedule.put(comp, new ArrayList<ExecutorDetails>());
+        }
+      }
+      
+      _slots = new LinkedList<Set<ExecutorDetails>>();
+      for (int i = 0; i < slotsToUse; i++) {
+        _slots.add(new HashSet<ExecutorDetails>());
+      }
+
+      int at = 0;
+      for (Entry<String, List<ExecutorDetails>> entry: _cluster.getNeedsSchedulingComponentToExecutors(td).entrySet()) {
+        LOG.debug("Scheduling for {}", entry.getKey());
+        if (_spreadToSchedule.containsKey(entry.getKey())) {
+          LOG.debug("Saving {} for spread...",entry.getKey());
+          _spreadToSchedule.get(entry.getKey()).addAll(entry.getValue());
+        } else {
+          for (ExecutorDetails ed: entry.getValue()) {
+            LOG.debug("Assigning {} {} to slot {}", new Object[]{entry.getKey(), ed, at});
+            _slots.get(at).add(ed);
+            at++;
+            if (at >= _slots.size()) {
+              at = 0;
+            }
+          }
+        }
+      }
+      _lastSlot = _slots.get(_slots.size() - 1);
+    }
+    
+    /**
+     * Assign a slot to the given node.
+     * @param n the node to assign a slot to.
+     * @return true if there are more slots to assign else false.
+     */
+    public boolean assignSlotTo(Node n) {
+      if (_slots.isEmpty()) {
+        return false;
+      }
+      Set<ExecutorDetails> slot = _slots.pop();
+      if (slot == _lastSlot) {
+        //The last slot fill it up
+        for (Entry<String, List<ExecutorDetails>> entry: _spreadToSchedule.entrySet()) {
+          if (entry.getValue().size() > 0) {
+            slot.addAll(entry.getValue());
+          }
+        }
+      } else {
+        String nodeId = n.getId();
+        Set<String> nodeComps = _nodeToComps.get(nodeId);
+        if (nodeComps == null) {
+          nodeComps = new HashSet<String>();
+          _nodeToComps.put(nodeId, nodeComps);
+        }
+        for (Entry<String, List<ExecutorDetails>> entry: _spreadToSchedule.entrySet()) {
+          if (entry.getValue().size() > 0) {
+            String comp = entry.getKey();
+            if (!nodeComps.contains(comp)) {
+              nodeComps.add(comp);
+              slot.add(entry.getValue().remove(0));
+            }
+          }
+        }
+      }
+      n.assign(_topId, slot, _cluster);
+      return !_slots.isEmpty();
+    }
+  }
+  
+  private static final Logger LOG = LoggerFactory.getLogger(NodePool.class);
+  /**
+   * Initialize the pool.
+   * @param cluster the cluster
+   * @param nodeIdToNode the mapping of node id to nodes
+   */
+  public void init(Cluster cluster, Map<String, Node> nodeIdToNode) {
+    _cluster = cluster;
+    _nodeIdToNode = nodeIdToNode;
+  }
+  
+  /**
+   * Add a topology to the pool
+   * @param td the topology to add.
+   */
+  public abstract void addTopology(TopologyDetails td);
+  
+  /**
+   * Check if this topology can be added to this pool
+   * @param td the topology
+   * @return true if it can else false
+   */
+  public abstract boolean canAdd(TopologyDetails td);
+  
+  /**
+   * @return the number of nodes that are available to be taken
+   */
+  public abstract int slotsAvailable();
+  
+  /**
+   * Take nodes from this pool that can fulfill possibly up to the
+   * slotsNeeded
+   * @param slotsNeeded the number of slots that are needed.
+   * @return a Collection of nodes with the removed nodes in it.  
+   * This may be empty, but should not be null.
+   */
+  public abstract Collection<Node> takeNodesBySlots(int slotsNeeded);
+
+  /**
+   * Get the number of nodes and slots this would provide to get the slots needed
+   * @param slots the number of slots needed
+   * @return the number of nodes and slots that would be returned.
+   */
+  public abstract NodeAndSlotCounts getNodeAndSlotCountIfSlotsWereTaken(int slots);
+  
+  /**
+   * @return the number of nodes that are available to be taken
+   */
+  public abstract int nodesAvailable();
+  
+  /**
+   * Take up to nodesNeeded from this pool
+   * @param nodesNeeded the number of nodes that are needed.
+   * @return a Collection of nodes with the removed nodes in it.  
+   * This may be empty, but should not be null.
+   */
+  public abstract Collection<Node> takeNodes(int nodesNeeded);
+  
+  /**
+   * Reschedule any topologies as needed.
+   * @param lesserPools pools that may be used to steal nodes from.
+   */
+  public abstract void scheduleAsNeeded(NodePool ... lesserPools);
+  
+  public static int slotsAvailable(NodePool[] pools) {
+    int slotsAvailable = 0;
+    for (NodePool pool: pools) {
+      slotsAvailable += pool.slotsAvailable();
+    }
+    return slotsAvailable;
+  }
+  
+  public static int nodesAvailable(NodePool[] pools) {
+    int nodesAvailable = 0;
+    for (NodePool pool: pools) {
+      nodesAvailable += pool.nodesAvailable();
+    }
+    return nodesAvailable;
+  }
+  
+  public static Collection<Node> takeNodesBySlot(int slotsNeeded,NodePool[] pools) {
+    LOG.debug("Trying to grab {} free slots from {}",slotsNeeded, pools);
+    HashSet<Node> ret = new HashSet<Node>();
+    for (NodePool pool: pools) {
+      Collection<Node> got = pool.takeNodesBySlots(slotsNeeded);
+      ret.addAll(got);
+      slotsNeeded -= Node.countFreeSlotsAlive(got);
+      LOG.debug("Got {} nodes so far need {} more slots",ret.size(),slotsNeeded);
+      if (slotsNeeded <= 0) {
+        break;
+      }
+    }
+    return ret;
+  }
+  
+  public static Collection<Node> takeNodes(int nodesNeeded,NodePool[] pools) {
+    LOG.debug("Trying to grab {} free nodes from {}",nodesNeeded, pools);
+    HashSet<Node> ret = new HashSet<Node>();
+    for (NodePool pool: pools) {
+      Collection<Node> got = pool.takeNodes(nodesNeeded);
+      ret.addAll(got);
+      nodesNeeded -= got.size();
+      LOG.debug("Got {} nodes so far need {} more nodes", ret.size(), nodesNeeded);
+      if (nodesNeeded <= 0) {
+        break;
+      }
+    }
+    return ret;
+  }
+
+  public static int getNodeCountIfSlotsWereTaken(int slots,NodePool[] pools) {
+    LOG.debug("How many nodes to get {} slots from {}",slots, pools);
+    int total = 0;
+    for (NodePool pool: pools) {
+      NodeAndSlotCounts ns = pool.getNodeAndSlotCountIfSlotsWereTaken(slots);
+      total += ns._nodes;
+      slots -= ns._slots;
+      LOG.debug("Found {} nodes so far {} more slots needed", total, slots);
+      if (slots <= 0) {
+        break;
+      }
+    }    
+    return total;
+  }
+}


[09/14] STORM-216: Added Authentication and Authorization.

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
----------------------------------------------------------------------
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 class Nimbus {
 
   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 class Nimbus {
 
     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 @@ public class Nimbus {
       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 @@ public class Nimbus {
       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 @@ public class Nimbus {
       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 @@ public class Nimbus {
       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 @@ public class Nimbus {
       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 @@ public class Nimbus {
       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 @@ public class Nimbus {
       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 @@ public class Nimbus {
       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 @@ public class Nimbus {
       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 @@ public class Nimbus {
       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 @@ public class Nimbus {
       if (result.ite != null) {
         throw result.ite;
       }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      return;
+    }
+
+    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 org.apache.thrift.TException
+    public String beginFileUpload() throws AuthorizationException, org.apache.thrift.TException
     {
       send_beginFileUpload();
       return recv_beginFileUpload();
@@ -331,17 +386,20 @@ public class Nimbus {
       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 @@ public class Nimbus {
       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 @@ public class Nimbus {
       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 @@ public class Nimbus {
       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 @@ public class Nimbus {
       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 @@ public class Nimbus {
       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 @@ public class Nimbus {
       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 @@ public class Nimbus {
       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 @@ public class Nimbus {
       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 @@ public class Nimbus {
       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 @@ public class Nimbus {
       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 @@ public class Nimbus {
       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 @@ public class Nimbus {
       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 @@ public class Nimbus {
       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 @@ public class Nimbus {
       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 @@ public class Nimbus {
         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 @@ public class Nimbus {
         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 @@ public class Nimbus {
         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 @@ public class Nimbus {
         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 @@ public class Nimbus {
         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 @@ public class Nimbus {
         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 @@ public class Nimbus {
         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 class Nimbus {
       }
     }
 
+    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 @@ public class Nimbus {
         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 @@ public class Nimbus {
         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 @@ public class Nimbus {
         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 @@ public class Nimbus {
         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 @@ public class Nimbus {
         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 @@ public class Nimbus {
         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 @@ public class Nimbus {
         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 @@ public class Nimbus {
         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 @@ public class Nimbus {
         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 @@ public class Nimbus {
         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 @@ public class Nimbus {
         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 @@ public class Nimbus {
       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 @@ public class Nimbus {
           result.e = e;
         } catch (InvalidTopologyException ite) {
           result.ite = ite;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
         }
         return result;
       }
@@ -1264,6 +1390,8 @@ public class Nimbus {
           result.e = e;
         } catch (InvalidTopologyException ite) {
           result.ite = ite;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
         }
         return result;
       }
@@ -1284,6 +1412,8 @@ public class Nimbus {
           iface.killTopology(args.name);
         } catch (NotAliveException e) {
           result.e = e;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
         }
         return result;
       }
@@ -1304,6 +1434,8 @@ public class Nimbus {
           iface.killTopologyWithOpts(args.name, args.options);
         } catch (NotAliveException e) {
           result.e = e;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
         }
         return result;
       }
@@ -1324,6 +1456,8 @@ public class Nimbus {
           iface.activate(args.name);
         } catch (NotAliveException e) {
           result.e = e;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
         }
         return result;
       }
@@ -1344,6 +1478,8 @@ public class Nimbus {
           iface.deactivate(args.name);
         } catch (NotAliveException e) {
           result.e = e;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
         }
         return result;
       }
@@ -1366,6 +1502,32 @@ public class Nimbus {
           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 @@ public class Nimbus {
 
       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 @@ public class Nimbus {
 
       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 @@ public class Nimbus {
 
       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 @@ public class Nimbus {
 
       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 @@ public class Nimbus {
 
       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 @@ public class Nimbus {
 
       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 @@ public class Nimbus {
 
       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 @@ public class Nimbus {
           result.success = iface.getTopologyInfo(args.id);
         } catch (NotAliveException e) {
           result.e = e;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
         }
         return result;
       }
@@ -1518,6 +1710,8 @@ public class Nimbus {
           result.success = iface.getTopologyConf(args.id);
         } catch (NotAliveException e) {
           result.e = e;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
         }
         return result;
       }
@@ -1538,6 +1732,8 @@ public class Nimbus {
           result.success = iface.getTopology(args.id);
         } catch (NotAliveException e) {
           result.e = e;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
         }
         return result;
       }
@@ -1558,6 +1754,8 @@ public class Nimbus {
           result.success = iface.getUserTopology(args.id);
         } catch (NotAliveException e) {
           result.e = e;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
         }
         return result;
       }
@@ -2151,14 +2349,17 @@ public class Nimbus {
 
     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 @@ public class Nimbus {
             return E;
           case 2: // ITE
             return ITE;
+          case 3: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -2225,6 +2428,8 @@ public class Nimbus {
           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 class Nimbus {
 
     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 @@ public class Nimbus {
       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 class Nimbus {
     public void clear() {
       this.e = null;
       this.ite = null;
+      this.aze = null;
     }
 
     public AlreadyAliveException get_e() {
@@ -2309,6 +2520,29 @@ public class Nimbus {
       }
     }
 
+    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 @@ public class Nimbus {
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -2338,6 +2580,9 @@ public class Nimbus {
       case ITE:
         return get_ite();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -2353,6 +2598,8 @@ public class Nimbus {
         return is_set_e();
       case ITE:
         return is_set_ite();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -2388,6 +2635,15 @@ public class Nimbus {
           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 @@ public class Nimbus {
       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 @@ public class Nimbus {
           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 @@ public class Nimbus {
               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 @@ public class Nimbus {
         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 @@ public class Nimbus {
         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 @@ public class Nimbus {
 
     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 @@ public class Nimbus {
             return E;
           case 2: // ITE
             return ITE;
+          case 3: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -3294,6 +3590,8 @@ public class Nimbus {
           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 class Nimbus {
 
     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 @@ public class Nimbus {
       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 class Nimbus {
     public void clear() {
       this.e = null;
       this.ite = null;
+      this.aze = null;
     }
 
     public AlreadyAliveException get_e() {
@@ -3378,6 +3682,29 @@ public class Nimbus {
       }
     }
 
+    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 @@ public class Nimbus {
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -3407,6 +3742,9 @@ public class Nimbus {
       case ITE:
         return get_ite();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -3422,6 +3760,8 @@ public class Nimbus {
         return is_set_e();
       case ITE:
         return is_set_ite();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -3457,6 +3797,15 @@ public class Nimbus {
           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 @@ public class Nimbus {
       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 @@ public class Nimbus {
           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 @@ public class Nimbus {
               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 @@ public class Nimbus {
         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 @@ public class Nimbus {
         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 @@ public class Nimbus {
     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 @@ public class Nimbus {
         switch(fieldId) {
           case 1: // E
             return E;
+          case 2: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -3982,6 +4371,8 @@ public class Nimbus {
       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 class Nimbus {
     }
 
     public killTopology_result(
-      NotAliveException e)
+      NotAliveException e,
+      AuthorizationException aze)
     {
       this();
       this.e = e;
+      this.aze = aze;
     }
 
     /**
@@ -4003,6 +4396,9 @@ public class Nimbus {
       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 @@ public class Nimbus {
     @Override
     public void clear() {
       this.e = null;
+      this.aze = null;
     }
 
     public NotAliveException get_e() {
@@ -4037,6 +4434,29 @@ public class Nimbus {
       }
     }
 
+    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 @@ public class Nimbus {
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -4055,6 +4483,9 @@ public class Nimbus {
       case E:
         return get_e();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -4068,6 +4499,8 @@ public class Nimbus {
       switch (field) {
       case E:
         return is_set_e();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -4094,6 +4527,15 @@ public class Nimbus {
           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 @@ public class Nimbus {
       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 @@ public class Nimbus {
           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 @@ public class Nimbus {
               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 @@ public class Nimbus {
         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 @@ public class Nimbus {
         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 @@ public class Nimbus {
     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 @@ public class Nimbus {
         switch(fieldId) {
           case 1: // E
             return E;
+          case 2: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -4678,6 +5160,8 @@ public class Nimbus {
       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 class Nimbus {
     }
 
     public killTopologyWithOpts_result(
-      NotAliveException e)
+      NotAliveException e,
+      AuthorizationException aze)
     {
       this();
       this.e = e;
+      this.aze = aze;
     }
 
     /**
@@ -4699,6 +5185,9 @@ public class Nimbus {
       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 @@ public class Nimbus {
     @Override
     public void clear() {
       this.e = null;
+      this.aze = null;
     }
 
     public NotAliveException get_e() {
@@ -4733,6 +5223,29 @@ public class Nimbus {
       }
     }
 
+    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 @@ public class Nimbus {
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -4751,6 +5272,9 @@ public class Nimbus {
       case E:
         return get_e();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -4764,6 +5288,8 @@ public class Nimbus {
       switch (field) {
       case E:
         return is_set_e();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -4790,6 +5316,15 @@ public class Nimbus {
           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 @@ public class Nimbus {
       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 @@ public class Nimbus {
           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,7 +5398,15 @@ public class Nimbus {
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
             }
             break;
-          default:
+          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);
         }
         iprot.readFieldEnd();
@@ -4864,6 +5422,10 @@ public class Nimbus {
         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 @@ public class Nimbus {
         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 @@ public class Nimbus {
     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 @@ public class Nimbus {
         switch(fieldId) {
           case 1: // E
             return E;
+          case 2: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -5280,6 +5855,8 @@ public class Nimbus {
       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 class Nimbus {
     }
 
     public activate_result(
-      NotAliveException e)
+      NotAliveException e,
+      AuthorizationException aze)
     {
       this();
       this.e = e;
+      this.aze = aze;
     }
 
     /**
@@ -5301,6 +5880,9 @@ public class Nimbus {
       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 @@ public class Nimbus {
     @Override
     public void clear() {
       this.e = null;
+      this.aze = null;
     }
 
     public NotAliveException get_e() {
@@ -5335,6 +5918,29 @@ public class Nimbus {
       }
     }
 
+    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 @@ public class Nimbus {
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -5353,6 +5967,9 @@ public class Nimbus {
       case E:
         return get_e();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -5366,6 +5983,8 @@ public class Nimbus {
       switch (field) {
       case E:
         return is_set_e();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -5392,6 +6011,15 @@ public class Nimbus {
           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 @@ public class Nimbus {
       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 @@ public class Nimbus {
           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 @@ public class Nimbus {
               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 @@ public class Nimbus {
         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 @@ public class Nimbus {
         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 @@ public class Nimbus {
     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 @@ public class Nimbus {
         switch(fieldId) {
           case 1: // E
             return E;
+          case 2: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -5882,6 +6550,8 @@ public class Nimbus {
       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 class Nimbus {
     }
 
     public deactivate_result(
-      NotAliveException e)
+      NotAliveException e,
+      AuthorizationException aze)
     {
       this();
       this.e = e;
+      this.aze = aze;
     }
 
     /**
@@ -5903,6 +6575,9 @@ public class Nimbus {
       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 @@ public class Nimbus {
     @Override
     public void clear() {
       this.e = null;
+      this.aze = null;
     }
 
     public NotAliveException get_e() {
@@ -5937,6 +6613,29 @@ public class Nimbus {
       }
     }
 
+    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 @@ public class Nimbus {
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -5955,6 +6662,9 @@ public class Nimbus {
       case E:
         return get_e();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -5968,6 +6678,8 @@ public class Nimbus {
       switch (field) {
       case E:
         return is_set_e();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -5994,6 +6706,15 @@ public class Nimbus {
           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 @@ public class Nimbus {
       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 @@ public class Nimbus {
           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 @@ public class Nimbus {
               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 @@ public class Nimbus {
         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 @@ public class Nimbus {
         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 @@ public class Nimbus {
 
     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 @@ public class Nimbus {
             return E;
           case 2: // ITE
             return ITE;
+          case 3: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -6585,6 +7346,8 @@ public class Nimbus {
           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 class Nimbus {
 
     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 @@ public class Nimbus {
       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 class Nimbus {
     public void clear() {
       this.e = null;
       this.ite = null;
+      this.aze = null;
     }
 
     public NotAliveException get_e() {
@@ -6669,6 +7438,29 @@ public class Nimbus {
       }
     }
 
+    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 @@ public class Nimbus {
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -6698,6 +7498,9 @@ public class Nimbus {
       case ITE:
         return get_ite();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -6713,6 +7516,8 @@ public class Nimbus {
         return is_set_e();
       case ITE:
         return is_set_ite();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -6748,6 +7553,15 @@ public class Nimbus {
           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 @@ public class Nimbus {
       if (present_ite)
         builder.append(ite);
 
+      boolean present_aze =

<TRUNCATED>

[12/14] STORM-216: Added Authentication and Authorization.

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/clj/backtype/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/worker.clj b/storm-core/src/clj/backtype/storm/daemon/worker.clj
index 921c259..265ed4b 100644
--- a/storm-core/src/clj/backtype/storm/daemon/worker.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/worker.clj
@@ -20,6 +20,9 @@
   (:import [java.util.concurrent Executors])
   (:import [backtype.storm.messaging TransportFactory])
   (:import [backtype.storm.messaging IContext IConnection])
+  (:import [backtype.storm.security.auth AuthUtils])
+  (:import [javax.security.auth Subject])
+  (:import [java.security PrivilegedExceptionAction])
   (:gen-class))
 
 (bootstrap)
@@ -106,23 +109,37 @@
               (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)]
-    (fn [^KryoTupleSerializer serializer tuple-batch]
-      (let [local (ArrayList.)
-            remote (ArrayList.)]
-        (fast-list-iter [[task tuple :as pair] tuple-batch]
-          (if (local-tasks task)
-            (.add local pair)
-            (.add remote pair)
-            ))
-        (local-transfer local)
-        ;; not using map because the lazy seq shows up in perf profiles
-        (let [serialized-pairs (fast-list-for [[task ^TupleImpl tuple] remote] [task (.serialize serializer tuple)])]
-          (disruptor/publish transfer-queue serialized-pairs)
-          )))))
+        ^DisruptorQueue transfer-queue (:transfer-queue worker)
+        try-serialize-local ((:conf worker) TOPOLOGY-TESTING-ALWAYS-TRY-SERIALIZE)
+        transfer-fn
+          (fn [^KryoTupleSerializer serializer tuple-batch]
+            (let [local (ArrayList.)
+                  remote (ArrayList.)]
+              (fast-list-iter [[task tuple :as pair] tuple-batch]
+                (if (local-tasks task)
+                  (.add local pair)
+                  (.add remote pair)
+              ))
+              (local-transfer local)
+              ;; not using map because the lazy seq shows up in perf profiles
+              (let [serialized-pairs (fast-list-for [[task ^TupleImpl tuple] remote] [task (.serialize serializer tuple)])]
+                (disruptor/publish transfer-queue serialized-pairs)
+              )))]
+    (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
@@ -164,21 +181,17 @@
                        (halt-process! 20 "Error when processing an event")
                        )))
 
-(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 (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
@@ -197,6 +210,7 @@
       :system-topology (system-topology! storm-conf topology)
       :heartbeat-timer (mk-halting-timer)
       :refresh-connections-timer (mk-halting-timer)
+      :refresh-credentials-timer (mk-halting-timer)
       :refresh-active-timer (mk-halting-timer)
       :executor-heartbeat-timer (mk-halting-timer)
       :user-timer (mk-halting-timer)
@@ -219,7 +233,31 @@
       :user-shared-resources (mk-user-resources <>)
       :transfer-local-fn (mk-transfer-local-fn <>)
       :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 (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))
@@ -336,6 +374,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
@@ -350,7 +393,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
@@ -363,13 +416,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)
@@ -400,6 +452,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))
@@ -423,19 +476,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]

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/clj/backtype/storm/testing.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/testing.clj b/storm-core/src/clj/backtype/storm/testing.clj
index 7bbe238..ec4de87 100644
--- a/storm-core/src/clj/backtype/storm/testing.clj
+++ b/storm-core/src/clj/backtype/storm/testing.clj
@@ -188,10 +188,16 @@
 
 (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,8 +210,8 @@
                   supervisors
                   workers) ; because a worker may already be dead
         ]
-    (while-timeout TEST-TIMEOUT-MS (not (every? (memfn waiting?) daemons))
-      (Thread/sleep 10)
+    (while-timeout TEST-TIMEOUT-MS (not (every? (memfn waiting?) daemons))      
+      (Thread/sleep (rand-int 20))
 ;;      (doseq [d daemons]
 ;;        (if-not ((memfn waiting?) d)
 ;;          (println d)))
@@ -280,7 +286,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))
       )))
 
@@ -471,6 +479,9 @@
     
     
     (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))
 
@@ -586,7 +597,7 @@
           ;; (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))
+          (Thread/sleep (rand-int 200)))
         (reset! (:last-spout-emit tracked-topology) target)
         )))
 
@@ -620,3 +631,9 @@
                   (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#)))))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/clj/backtype/storm/testing4j.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/testing4j.clj b/storm-core/src/clj/backtype/storm/testing4j.clj
index 5d44604..d5f6550 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])
@@ -151,4 +151,4 @@
        (let [stream (or (.getStream param) Utils/DEFAULT_STREAM_ID)
              component (or (.getComponent param) "component")
              fields (.getFields param)]
-         (test-tuple values :stream stream :component component :fields fields)))))
\ No newline at end of file
+         (test-tuple values :stream stream :component component :fields fields)))))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/clj/backtype/storm/thrift.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/thrift.clj b/storm-core/src/clj/backtype/storm/thrift.clj
index ec8ce5d..c2eb0c7 100644
--- a/storm-core/src/clj/backtype/storm/thrift.clj
+++ b/storm-core/src/clj/backtype/storm/thrift.clj
@@ -18,13 +18,12 @@
   (:import [backtype.storm.generated JavaObject Grouping Nimbus StormTopology StormTopology$_Fields 
     Bolt Nimbus$Client Nimbus$Iface 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])
   )
 
@@ -65,10 +64,10 @@
 
 (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)
+  (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]

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/clj/backtype/storm/timer.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/timer.clj b/storm-core/src/clj/backtype/storm/timer.clj
index 78b1f1c..5eef1d3 100644
--- a/storm-core/src/clj/backtype/storm/timer.clj
+++ b/storm-core/src/clj/backtype/storm/timer.clj
@@ -23,7 +23,7 @@
 ;; The timer defined in this file is very similar to java.util.Timer, except it integrates with
 ;; Storm's time simulation capabilities. This lets us test code that does asynchronous work on the timer thread
 
-(defnk mk-timer [:kill-fn (fn [& _] )]
+(defnk mk-timer [:kill-fn (fn [& _] ) :thread-name nil]
   (let [queue (PriorityQueue. 10
                               (reify Comparator
                                 (compare [this o1 o2]
@@ -35,35 +35,37 @@
         active (atom true)
         lock (Object.)
         notifier (Semaphore. 0)
-        timer-thread (Thread.
-                      (fn []
-                        (while @active
-                          (try
-                            (let [[time-millis _ _ :as elem] (locking lock (.peek queue))]
-                              (if (and elem (>= (current-time-millis) time-millis))
-                                ;; imperative to not run the function inside the timer lock
-                                ;; otherwise, it's possible to deadlock if function deals with other locks
-                                ;; (like the submit lock)
-                                (let [afn (locking lock (second (.poll queue)))]
-                                  (afn))
-                                (if time-millis ;; if any events are scheduled
-                                  ;; sleep until event generation
-                                  ;; note that if any recurring events are scheduled then we will always go through
-                                  ;; this branch, sleeping only the exact necessary amount of time
-                                  (Time/sleep (- time-millis (current-time-millis)))
-                                  ;; else poll to see if any new event was scheduled
-                                  ;; this is in essence the response time for detecting any new event schedulings when
-                                  ;; there are no scheduled events
-                                  (Time/sleep 1000))
-                                ))
-                            (catch Throwable t
-                              ;; because the interrupted exception can be wrapped in a runtimeexception
-                              (when-not (exception-cause? InterruptedException t)
-                                (kill-fn t)
-                                (reset! active false)
-                                (throw t))
-                              )))
-                        (.release notifier)))]
+        thread-fn (fn []
+                    (while @active
+                      (try
+                        (let [[time-millis _ _ :as elem] (locking lock (.peek queue))]
+                          (if (and elem (>= (current-time-millis) time-millis))
+                            ;; imperative to not run the function inside the timer lock
+                            ;; otherwise, it's possible to deadlock if function deals with other locks
+                            ;; (like the submit lock)
+                            (let [afn (locking lock (second (.poll queue)))]
+                              (afn))
+                            (if time-millis ;; if any events are scheduled
+                              ;; sleep until event generation
+                              ;; note that if any recurring events are scheduled then we will always go through
+                              ;; this branch, sleeping only the exact necessary amount of time
+                              (Time/sleep (- time-millis (current-time-millis)))
+                              ;; else poll to see if any new event was scheduled
+                              ;; this is in essence the response time for detecting any new event schedulings when
+                              ;; there are no scheduled events
+                              (Time/sleep 1000))
+                            ))
+                        (catch Throwable t
+                          ;; because the interrupted exception can be wrapped in a runtimeexception
+                          (when-not (exception-cause? InterruptedException t)
+                            (kill-fn t)
+                            (reset! active false)
+                            (throw t))
+                          )))
+                    (.release notifier))
+        timer-thread (if (clojure.string/blank? thread-name)
+                       (Thread. thread-fn)
+                       (Thread. thread-fn thread-name))]
     (.setDaemon timer-thread true)
     (.setPriority timer-thread Thread/MAX_PRIORITY)
     (.start timer-thread)

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/ui/core.clj b/storm-core/src/clj/backtype/storm/ui/core.clj
index ad1a038..6d7cd44 100644
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@ -21,14 +21,16 @@
   (:use [backtype.storm.ui helpers])
   (:use [backtype.storm.daemon [common :only [ACKER-COMPONENT-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 [java.io File])
+  (:import [backtype.storm.security.auth AuthUtils])
+  (:import [java.io File PrintWriter StringWriter])
+  (:import [java.net URLDecoder])
   (:require [compojure.route :as route]
             [compojure.handler :as handler]
             [ring.util.response :as resp]
@@ -51,6 +53,7 @@
 (def tips
   "Defines a mapping of help texts for elements of the UI pages."
   {:sys-stats "Use this to toggle inclusion of storm system components."
+   :user "This should be you."
    :version (str "The version of storm installed on the UI node. (Hopefully, "
                  "this is the same on all storm nodes!)")
    :nimbus-uptime (str "The duration the current Nimbus instance has been "
@@ -65,9 +68,13 @@
    :name "The name given to the topology by when it was submitted."
    :name-link "Click the name to view the Topology's information."
    :topo-id "The unique ID given to a Topology each time it is launched."
+   :owner "The user that submitted the Topology, if authentication is enabled."
    :status "The status can be one of ACTIVE, INACTIVE, KILLED, or REBALANCING."
    :topo-uptime "The time since the Topology was submitted."
    :num-workers "The number of Workers (processes)."
+   :scheduler-info (str "This shows information from the scheduler about the "
+                        "latest attempt to schedule the Topology on the "
+                        "cluster.")
    :sup-id (str "A unique identifier given to a Supervisor when it joins the "
                 "cluster.")
    :sup-host (str "The hostname reported by the remote host. (Note that this "
@@ -112,27 +119,33 @@
 
 (defn mk-system-toggle-button [include-sys?]
   [:p {:class "js-only"}
-   [:span.tip.right {:title (:sys-stats tips)}
-    [:input {:type "button"
+    [:span.tip.right {:title (:sys-stats tips)}
+     [:input {:type "button"
              :value (str (if include-sys? "Hide" "Show") " System Stats")
              :onclick "toggleSys()"}]]])
 
-(defn ui-template [body]
-  (html4
-   [:head
-    [:title "Storm UI"]
-    (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/bootstrap-twipsy.js")
-    (include-js "/js/script.js")
-    ]
-   [:body
-    [:h1 (link-to "/" "Storm UI")]
-    (seq body)
-    ]))
+(defn ui-template
+  ([body] (ui-template body nil))
+  ([body user]
+    (html4
+     [:head
+      [:title "Storm UI"]
+      (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/bootstrap-twipsy.js")
+      (include-js "/js/script.js")
+      ]
+     [:body
+      (concat
+        (when (not (blank? user))
+          [[:div.ui-user
+            [:p [:span.tip.below {:title (:user tips)} "User: " user]]]])
+        [[:h1 (link-to "/" "Storm UI")]]
+        (seq body))
+      ])))
 
 (defn read-storm-version []
   (let [storm-home (System/getProperty "storm.home")
@@ -190,6 +203,8 @@
                          :title (str (:name tips) " " (:name-link tips))}}
     {:text "Id" :attr {:class "tip right"
                        :title (:topo-id tips)}}
+    {:text "Owner" :attr {:class "tip above"
+                          :title (:owner tips)}}
     {:text "Status" :attr {:class "tip above"
                            :title (:status tips)}}
     {:text "Uptime" :attr {:class "tip above"
@@ -199,17 +214,21 @@
     {:text "Num executors" :attr {:class "tip above"
                                   :title (:num-execs tips)}}
     {:text "Num tasks" :attr {:class "tip above"
-                              :title (:num-tasks tips)}}]
+                              :title (:num-tasks tips)}}
+    {:text "Scheduler Info" :attr {:class "tip left"
+                                   :title (:scheduler-info tips)}}]
    (for [^TopologySummary t summs]
      [(topology-link (.get_id t) (.get_name t))
       (escape-html (.get_id t))
+      (escape-html (.get_owner t))
       (.get_status t)
       (pretty-uptime-sec (.get_uptime_secs t))
       (.get_num_workers t)
       (.get_num_executors t)
       (.get_num_tasks t)
+      (.get_sched_status t)
       ])
-   :time-cols [3]
+   :time-cols [4]
    :sort-list "[[0,0]]"
    ))
 
@@ -421,6 +440,8 @@
                                  :title (:name tips)}}
             {:text "Id" :attr {:class "tip right"
                                :title (:topo-id tips)}}
+            {:text "Owner" :attr {:class "tip above"
+                                  :title (:owner tips)}}
             {:text "Status" :attr {:class "tip above"
                                    :title (:status tips)}}
             {:text "Uptime" :attr {:class "tip above"
@@ -430,14 +451,18 @@
             {:text "Num executors" :attr {:class "tip above"
                                           :title (:num-execs tips)}}
             {:text "Num tasks" :attr {:class "tip above"
-                                      :title (:num-tasks tips)}}]
+                                      :title (:num-tasks tips)}}
+            {:text "Scheduler Info" :attr {:class "tip left"
+                                           :title (:scheduler-info tips)}}]
            [[(escape-html (.get_name summ))
              (escape-html (.get_id summ))
+             (escape-html (.get_owner summ))
              (.get_status summ)
              (pretty-uptime-sec (.get_uptime_secs summ))
              (count workers)
              (count executors)
              (sum-tasks executors)
+             (.get_sched_status summ)
              ]]
            )))
 
@@ -519,9 +544,10 @@
 (defn component-link [storm-id id]
   (link-to (url-format "/topology/%s/component/%s" storm-id id) (escape-html id)))
 
-(defn worker-log-link [host port]
-  (link-to (url-format "http://%s:%s/log?file=worker-%s.log"
-              host (*STORM-CONF* LOGVIEWER-PORT) port) (str port)))
+(defn worker-log-link [host port topology-id]
+  (let [fname (logs-filename topology-id port)]
+    (link-to (url-format (str "http://%s:%s/log?file=%s")
+                host (*STORM-CONF* LOGVIEWER-PORT) fname) (str port))))
 
 (defn render-capacity [capacity]
   (let [capacity (nil-to-zero capacity)]
@@ -649,7 +675,30 @@
                          (StringEscapeUtils/escapeJavaScript name) "', '"
                          command "', " is-wait ", " default-wait ")")}])
 
-(defn topology-page [id window include-sys?]
+(defn- ui-actions-enabled? []
+  (= "true" (lower-case (*STORM-CONF* UI-ACTIONS-ENABLED))))
+
+(defn- topology-actions [id name status msg-timeout]
+  (if (ui-actions-enabled?)
+    (concat
+       [[:h2 {:class "js-only"} "Topology actions"]]
+       [[:p {:class "js-only"} (concat
+         [(topology-action-button id name "Activate" "activate" false 0 (= "INACTIVE" status))]
+         [(topology-action-button id name "Deactivate" "deactivate" false 0 (= "ACTIVE" status))]
+         [(topology-action-button id name "Rebalance" "rebalance" true msg-timeout (or (= "ACTIVE" status) (= "INACTIVE" status)))]
+         [(topology-action-button id name "Kill" "kill" true msg-timeout (not= "KILLED" status))]
+       )]] )
+    []))
+
+(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))]
+    (and (not (blank? user))
+         (some #(= % user) ui-users))))
+
+(defn topology-page [id window include-sys? user]
   (with-nimbus nimbus
     (let [window (if window window ":all-time")
           window-hint (window-hint window)
@@ -665,25 +714,24 @@
           status (.get_status summ)
           msg-timeout (topology-conf TOPOLOGY-MESSAGE-TIMEOUT-SECS)
           ]
-      (concat
-       [[:h2 "Topology summary"]]
-       [(topology-summary-table summ)]
-       [[:h2 {:class "js-only"} "Topology actions"]]
-       [[:p {:class "js-only"} (concat
-         [(topology-action-button id name "Activate" "activate" false 0 (= "INACTIVE" status))]
-         [(topology-action-button id name "Deactivate" "deactivate" false 0 (= "ACTIVE" status))]
-         [(topology-action-button id name "Rebalance" "rebalance" true msg-timeout (or (= "ACTIVE" status) (= "INACTIVE" status)))]
-         [(topology-action-button id name "Kill" "kill" true msg-timeout (not= "KILLED" status))]
-       )]]
-       [[:h2 "Topology stats"]]
-       (topology-stats-table id window (total-aggregate-stats spout-summs bolt-summs include-sys?))
-       [[:h2 "Spouts (" window-hint ")"]]
-       (spout-comp-table id spout-comp-summs (.get_errors summ) window include-sys?)
-       [[:h2 "Bolts (" window-hint ")"]]
-       (bolt-comp-table id bolt-comp-summs (.get_errors summ) window include-sys?)
-       [[:h2 "Topology Configuration"]]
-       (configuration-table topology-conf)
-       ))))
+      (if (or (blank? (*STORM-CONF* UI-FILTER))
+              (authorized-ui-user? user *STORM-CONF* topology-conf))
+        (concat
+          [[:h2 "Topology summary"]]
+          [(topology-summary-table summ)]
+          (topology-actions id name status msg-timeout)
+          [[:h2 "Topology stats"]]
+          (topology-stats-table id window (total-aggregate-stats spout-summs bolt-summs include-sys?))
+          [[:h2 "Spouts (" window-hint ")"]]
+          (spout-comp-table id spout-comp-summs (.get_errors summ) window include-sys?)
+          [[:h2 "Bolts (" window-hint ")"]]
+          (bolt-comp-table id bolt-comp-summs (.get_errors summ) window include-sys?)
+          [[:h2 "Topology Configuration"]]
+          (configuration-table topology-conf)
+          [(mk-system-toggle-button include-sys?)]
+        )
+
+        (unauthorized-user-html user)))))
 
 (defn component-task-summs [^TopologyInfo summ topology id]
   (let [spout-summs (filter (partial spout-summary? topology) (.get_executors summ))
@@ -781,7 +829,7 @@
      [(pretty-executor-info (.get_executor_info e))
       (pretty-uptime-sec (.get_uptime_secs e))
       (.get_host e)
-      (worker-log-link (.get_host e) (.get_port e))
+      (worker-log-link (.get_host e) (.get_port e) topology-id)
       (nil-to-zero (:emitted stats))
       (nil-to-zero (:transferred stats))
       (float-str (:complete-latencies stats))
@@ -896,7 +944,7 @@
      [(pretty-executor-info (.get_executor_info e))
       (pretty-uptime-sec (.get_uptime_secs e))
       (.get_host e)
-      (worker-log-link (.get_host e) (.get_port e))
+      (worker-log-link (.get_host e) (.get_port e) topology-id)
       (nil-to-zero (:emitted stats))
       (nil-to-zero (:transferred stats))
       (render-capacity (compute-executor-capacity e))
@@ -977,7 +1025,7 @@
      :sort-list "[[0,1]]"
      )))
 
-(defn component-page [topology-id component window include-sys?]
+(defn component-page [topology-id component window include-sys? user]
   (with-nimbus nimbus
     (let [window (if window window ":all-time")
           summ (.getTopologyInfo ^Nimbus$Client nimbus topology-id)
@@ -985,89 +1033,106 @@
           type (component-type topology component)
           summs (component-task-summs summ topology component)
           spec (cond (= type :spout) (spout-page window summ component summs include-sys?)
-                     (= type :bolt) (bolt-page window summ component summs include-sys?))]
-      (concat
-       [[:h2 "Component summary"]
-        (table [{:text "Id" :attr {:class "tip right"
-                                   :title (:comp-id tips)}}
-                {:text "Topology" :attr {:class "tip above"
-                                   :title (str (:name tips) " " (:name-link tips))}}
-                {:text "Executors" :attr {:class "tip above"
-                                   :title (:num-execs tips)}}
-                {:text "Tasks" :attr {:class "tip above"
-                               :title (:num-tasks tips)}}]
-               [[(escape-html component)
-                 (topology-link (.get_id summ) (.get_name summ))
-                 (count summs)
-                 (sum-tasks summs)
-                 ]])]
-       spec
-       [[:h2 "Errors"]
-        (errors-table (get (.get_errors summ) component))]
-       ))))
+                     (= type :bolt) (bolt-page window summ component summs include-sys?))
+          topology-conf (from-json (.getTopologyConf ^Nimbus$Client nimbus topology-id))]
+      (if (or (blank? (*STORM-CONF* UI-FILTER))
+              (authorized-ui-user? user *STORM-CONF* topology-conf))
+        (concat
+          [[:h2 "Component summary"]
+           (table [{:text "Id" :attr {:class "tip right"
+                                      :title (:comp-id tips)}}
+                   {:text "Topology" :attr {:class "tip above"
+                                      :title (str (:name tips) " " (:name-link tips))}}
+                   {:text "Executors" :attr {:class "tip above"
+                                      :title (:num-execs tips)}}
+                   {:text "Tasks" :attr {:class "tip above"
+                                  :title (:num-tasks tips)}}]
+                  [[(escape-html component)
+                    (topology-link (.get_id summ) (.get_name summ))
+                    (count summs)
+                    (sum-tasks summs)
+                    ]])]
+          spec
+          [[:h2 "Errors"]
+           (errors-table (get (.get_errors summ) component))
+           (mk-system-toggle-button include-sys?)])
+
+        (unauthorized-user-html user)))))
 
 (defn get-include-sys? [cookies]
   (let [sys? (get cookies "sys")
         sys? (if (or (nil? sys?) (= "false" (:value sys?))) false true)]
     sys?))
 
-(defroutes main-routes
-  (GET "/" [:as {cookies :cookies}]
-       (-> (main-page)
-           ui-template))
-  (GET "/topology/:id" [:as {cookies :cookies} id & m]
-       (let [include-sys? (get-include-sys? cookies)
-            id (url-decode id)]
-         (try
-           (-> (topology-page (url-decode id) (:window m) include-sys?)
-             (concat [(mk-system-toggle-button include-sys?)])
-             ui-template)
-           (catch Exception e (resp/redirect "/")))))
-  (GET "/topology/:id/component/:component" [:as {cookies :cookies} id component & m]
-       (let [include-sys? (get-include-sys? cookies)
-            id (url-decode id)
-            component (url-decode component)]
-         (-> (component-page id component (:window m) include-sys?)
-             (concat [(mk-system-toggle-button include-sys?)])
-             ui-template)))
-  (POST "/topology/:id/activate" [id]
-    (with-nimbus nimbus
-      (let [id (url-decode id)
-            tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
-            name (.get_name tplg)]
-        (.activate nimbus name)
-        (log-message "Activating topology '" name "'")))
-    (resp/redirect (str "/topology/" id)))
-  (POST "/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 "/topology/" id)))
-  (POST "/topology/:id/rebalance/:wait-time" [id wait-time]
-    (with-nimbus nimbus
-      (let [id (url-decode id)
-            tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
-            name (.get_name tplg)
-            options (RebalanceOptions.)]
-        (.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 "/topology/" id)))
-  (POST "/topology/:id/kill/:wait-time" [id wait-time]
-    (with-nimbus nimbus
-      (let [id (url-decode id)
-            tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
-            name (.get_name tplg)
-            options (KillOptions.)]
-        (.set_wait_secs options (Integer/parseInt wait-time))
-        (.killTopologyWithOpts nimbus name options)
-        (log-message "Killing topology '" name "' with wait time: " wait-time " secs")))
-    (resp/redirect (str "/topology/" id)))
-  (route/resources "/")
-  (route/not-found "Page not found"))
+(def http-creds-handler (AuthUtils/GetUiHttpCredentialsPlugin *STORM-CONF*))
+
+(if (ui-actions-enabled?)
+  (defroutes main-routes
+    (GET "/" [:as {servlet-request :servlet-request}]
+         (ui-template (main-page)
+                      (.getUserName http-creds-handler servlet-request)))
+    (GET "/topology/:id" [:as {:keys [cookies servlet-request]} id & m]
+         (let [include-sys? (get-include-sys? cookies)
+               user (.getUserName http-creds-handler servlet-request)]
+           (ui-template (topology-page (URLDecoder/decode id) (:window m) include-sys? user)
+                        user)))
+    (GET "/topology/:id/component/:component" [:as {:keys [cookies servlet-request]}
+                                               id component & m]
+         (let [include-sys? (get-include-sys? cookies)
+               user (.getUserName http-creds-handler servlet-request)]
+           (ui-template (component-page (URLDecoder/decode id) component (:window m) include-sys? user)
+                        user)))
+    (POST "/topology/:id/activate" [id]
+      (with-nimbus nimbus
+        (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus (URLDecoder/decode id))
+              name (.get_name tplg)]
+          (.activate nimbus name)
+          (log-message "Activating topology '" name "'")))
+      (resp/redirect (str "/topology/" id)))
+    (POST "/topology/:id/deactivate" [id]
+      (with-nimbus nimbus
+        (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus (URLDecoder/decode id))
+              name (.get_name tplg)]
+          (.deactivate nimbus name)
+          (log-message "Deactivating topology '" name "'")))
+      (resp/redirect (str "/topology/" id)))
+    (POST "/topology/:id/rebalance/:wait-time" [id wait-time]
+      (with-nimbus nimbus
+        (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus (URLDecoder/decode id))
+              name (.get_name tplg)
+              options (RebalanceOptions.)]
+          (.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 "/topology/" id)))
+    (POST "/topology/:id/kill/:wait-time" [id wait-time]
+      (with-nimbus nimbus
+        (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus (URLDecoder/decode id))
+              name (.get_name tplg)
+              options (KillOptions.)]
+          (.set_wait_secs options (Integer/parseInt wait-time))
+          (.killTopologyWithOpts nimbus name options)
+          (log-message "Killing topology '" name "' with wait time: " wait-time " secs")))
+      (resp/redirect (str "/topology/" id)))
+    (route/resources "/")
+    (route/not-found "Page not found"))
+
+  (defroutes main-routes
+    (GET "/" [:as {servlet-request :servlet-request}]
+         (ui-template (main-page)
+                      (.getUserName http-creds-handler servlet-request)))
+    (GET "/topology/:id" [:as {:keys [cookies servlet-request]} id & m]
+         (let [include-sys? (get-include-sys? cookies)
+               user (.getUserName http-creds-handler servlet-request)]
+           (ui-template (topology-page (URLDecoder/decode id) (:window m) include-sys? user) user)))
+    (GET "/topology/:id/component/:component" [:as {:keys [cookies servlet-request]}
+                                               id component & m]
+         (let [include-sys? (get-include-sys? cookies)
+               user (.getUserName http-creds-handler servlet-request)]
+           (ui-template (component-page (URLDecoder/decode id) component (:window m) include-sys? user)
+                        user)))
+    (route/resources "/")
+    (route/not-found "Page not found")))
 
 (defn exception->html [ex]
   (concat
@@ -1082,8 +1147,9 @@
       (handler request)
       (catch Exception ex
         (-> (resp/response (ui-template (exception->html ex)))
-          (resp/status 500)
-          (resp/content-type "text/html"))
+            (resp/status 500)
+            (resp/content-type "text/html"))
+        (log-error ex)
         ))))
 
 (def app
@@ -1091,7 +1157,19 @@
                     (wrap-reload '[backtype.storm.ui.core])
                     catch-errors)))
 
-(defn start-server! [] (run-jetty app {:port (Integer. (*STORM-CONF* UI-PORT))
-                                       :join? false}))
+(defn start-server! []
+  (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!))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/clj/backtype/storm/ui/helpers.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/ui/helpers.clj b/storm-core/src/clj/backtype/storm/ui/helpers.clj
index dcd4a21..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]))
 
@@ -147,3 +152,39 @@ $(\"table#%s\").each(function(i) { $(this).tablesorter({ sortList: %s, headers:
 (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)})

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/clj/backtype/storm/util.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/util.clj b/storm-core/src/clj/backtype/storm/util.clj
index 6714fcd..45b5682 100644
--- a/storm-core/src/clj/backtype/storm/util.clj
+++ b/storm-core/src/clj/backtype/storm/util.clj
@@ -24,12 +24,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]])
@@ -397,13 +400,16 @@
     (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)
@@ -463,6 +469,39 @@
         ))
       ))
 
+(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)))
 
@@ -888,3 +927,32 @@
                 (meta form))
               (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))))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/clj/backtype/storm/zookeeper.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/zookeeper.clj b/storm-core/src/clj/backtype/storm/zookeeper.clj
index ab3e0b2..2abd20c 100644
--- a/storm-core/src/clj/backtype/storm/zookeeper.clj
+++ b/storm-core/src/clj/backtype/storm/zookeeper.clj
@@ -81,12 +81,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?]
   ((complement nil?)
@@ -102,12 +103,12 @@
       (when-not force (throw e)))
     (catch Exception e (throw (wrap-in-runtime e)))))
 
-(defn mkdirs [^CuratorFramework zk ^String path]
+(defn mkdirs [^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
           ))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/clj/storm/trident/testing.clj
----------------------------------------------------------------------
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])

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/dev/drpc-simple-acl-test-scenario.yaml
----------------------------------------------------------------------
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"

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java
index 83ee232..07da267 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -20,6 +20,7 @@ package backtype.storm;
 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 @@ import java.util.Map;
  */
 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";
@@ -131,7 +139,13 @@ public class Config extends HashMap<String, Object> {
     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;
@@ -144,6 +158,13 @@ public class Config extends HashMap<String, Object> {
     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 
@@ -191,30 +212,72 @@ public class Config extends HashMap<String, Object> {
     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.
      */
@@ -222,12 +285,32 @@ public class Config extends HashMap<String, Object> {
     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.
@@ -318,6 +401,18 @@ public class Config extends HashMap<String, Object> {
     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";
@@ -336,6 +431,24 @@ public class Config extends HashMap<String, Object> {
     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";
@@ -348,36 +461,156 @@ public class Config extends HashMap<String, Object> {
     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.
@@ -392,7 +625,19 @@ public class Config extends HashMap<String, Object> {
     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;
@@ -403,7 +648,31 @@ public class Config extends HashMap<String, Object> {
      */
     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
@@ -412,7 +681,6 @@ public class Config extends HashMap<String, Object> {
     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.
@@ -420,7 +688,6 @@ public class Config extends HashMap<String, Object> {
     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
@@ -430,7 +697,6 @@ public class Config extends HashMap<String, Object> {
     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
@@ -439,7 +705,6 @@ public class Config extends HashMap<String, Object> {
     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.
      */
@@ -455,13 +720,35 @@ public class Config extends HashMap<String, Object> {
     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;
+
+    /**
      * How often this worker should heartbeat to the supervisor.
      */
     public static final String WORKER_HEARTBEAT_FREQUENCY_SECS = "worker.heartbeat.frequency.secs";
@@ -485,6 +772,19 @@ public class Config extends HashMap<String, Object> {
     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
@@ -663,6 +963,12 @@ public class Config extends HashMap<String, Object> {
     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.
@@ -750,10 +1056,34 @@ public class Config extends HashMap<String, Object> {
     /**
      * 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";
@@ -822,8 +1152,22 @@ public class Config extends HashMap<String, Object> {
      * 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);
     } 


[14/14] git commit: STORM-216: Added Authentication and Authorization.

Posted by bo...@apache.org.
STORM-216: Added Authentication and Authorization.

This is an upmerged version of https://github.com/yahoo/incubator-storm/tree/security


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

Branch: refs/heads/security
Commit: fe5f41aa8332700d3f98422cb7d986fc47289bcd
Parents: 4ac0bec
Author: Robert (Bobby) Evans <bo...@apache.org>
Authored: Wed May 21 11:03:11 2014 -0500
Committer: Robert (Bobby) Evans <bo...@apache.org>
Committed: Wed May 21 11:03:11 2014 -0500

----------------------------------------------------------------------
 .gitignore                                      |    2 +-
 SECURITY.md                                     |  327 +-
 bin/storm                                       |   15 +-
 conf/defaults.yaml                              |   36 +
 conf/jaas_digest.conf                           |    8 +-
 conf/jaas_kerberos.conf                         |   15 +
 pom.xml                                         |   20 +
 storm-core/pom.xml                              |  229 +-
 .../src/clj/backtype/storm/LocalCluster.clj     |    3 +
 storm-core/src/clj/backtype/storm/LocalDRPC.clj |    4 +-
 storm-core/src/clj/backtype/storm/bootstrap.clj |    5 +-
 storm-core/src/clj/backtype/storm/cluster.clj   |   96 +-
 .../storm/command/upload_credentials.clj        |   35 +
 storm-core/src/clj/backtype/storm/config.clj    |   41 +-
 .../src/clj/backtype/storm/daemon/common.clj    |   18 +-
 .../src/clj/backtype/storm/daemon/drpc.clj      |  164 +-
 .../src/clj/backtype/storm/daemon/executor.clj  |  116 +-
 .../src/clj/backtype/storm/daemon/logviewer.clj |  377 ++-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |  237 +-
 .../clj/backtype/storm/daemon/supervisor.clj    |  190 +-
 .../src/clj/backtype/storm/daemon/task.clj      |    7 +-
 .../src/clj/backtype/storm/daemon/worker.clj    |  135 +-
 storm-core/src/clj/backtype/storm/testing.clj   |   27 +-
 storm-core/src/clj/backtype/storm/testing4j.clj |    4 +-
 storm-core/src/clj/backtype/storm/thrift.clj    |   13 +-
 storm-core/src/clj/backtype/storm/timer.clj     |   62 +-
 storm-core/src/clj/backtype/storm/ui/core.clj   |  338 +-
 .../src/clj/backtype/storm/ui/helpers.clj       |   45 +-
 storm-core/src/clj/backtype/storm/util.clj      |   84 +-
 storm-core/src/clj/backtype/storm/zookeeper.clj |   19 +-
 storm-core/src/clj/storm/trident/testing.clj    |    1 +
 .../src/dev/drpc-simple-acl-test-scenario.yaml  |   11 +
 storm-core/src/jvm/backtype/storm/Config.java   |  368 ++-
 .../jvm/backtype/storm/ConfigValidation.java    |  156 +-
 .../src/jvm/backtype/storm/Constants.java       |    3 +-
 .../backtype/storm/ICredentialsListener.java    |   32 +
 .../src/jvm/backtype/storm/ILocalCluster.java   |    2 +
 .../src/jvm/backtype/storm/StormSubmitter.java  |  161 +-
 .../storm/drpc/DRPCInvocationsClient.java       |   91 +-
 .../src/jvm/backtype/storm/drpc/DRPCSpout.java  |  100 +-
 .../jvm/backtype/storm/drpc/ReturnResults.java  |   35 +-
 .../storm/generated/AuthorizationException.java |  328 ++
 .../backtype/storm/generated/Credentials.java   |  373 +++
 .../storm/generated/DistributedRPC.java         |  110 +-
 .../generated/DistributedRPCInvocations.java    |  352 +-
 .../jvm/backtype/storm/generated/Nimbus.java    | 3006 +++++++++++++++++-
 .../backtype/storm/generated/SubmitOptions.java |   98 +-
 .../backtype/storm/generated/TopologyInfo.java  |  192 +-
 .../storm/generated/TopologySummary.java        |  192 +-
 .../jvm/backtype/storm/scheduler/Cluster.java   |   13 +
 .../scheduler/multitenant/DefaultPool.java      |  219 ++
 .../storm/scheduler/multitenant/FreePool.java   |  125 +
 .../scheduler/multitenant/IsolatedPool.java     |  346 ++
 .../multitenant/MultitenantScheduler.java       |   98 +
 .../storm/scheduler/multitenant/Node.java       |  322 ++
 .../storm/scheduler/multitenant/NodePool.java   |  296 ++
 .../backtype/storm/security/auth/AuthUtils.java |  165 +-
 .../auth/DefaultHttpCredentialsPlugin.java      |   83 +
 .../security/auth/DefaultPrincipalToLocal.java  |   43 +
 .../storm/security/auth/IAutoCredentials.java   |   55 +
 .../security/auth/ICredentialsRenewer.java      |   40 +
 .../security/auth/IHttpCredentialsPlugin.java   |   50 +
 .../storm/security/auth/IPrincipalToLocal.java  |   41 +
 .../storm/security/auth/ITransportPlugin.java   |   14 +-
 .../security/auth/KerberosPrincipalToLocal.java |   45 +
 .../storm/security/auth/ReqContext.java         |   10 +-
 .../security/auth/SaslTransportPlugin.java      |   45 +-
 .../security/auth/SimpleTransportPlugin.java    |   61 +-
 .../security/auth/SingleUserPrincipal.java      |   56 +
 .../storm/security/auth/TBackoffConnect.java    |   77 +
 .../storm/security/auth/ThriftClient.java       |   85 +-
 .../security/auth/ThriftConnectionType.java     |   77 +
 .../storm/security/auth/ThriftServer.java       |   19 +-
 .../auth/authorizer/DRPCAuthorizerBase.java     |   46 +
 .../authorizer/DRPCSimpleACLAuthorizer.java     |  157 +
 .../auth/authorizer/DenyAuthorizer.java         |    4 +-
 .../auth/authorizer/NoopAuthorizer.java         |    6 +-
 .../auth/authorizer/SimpleACLAuthorizer.java    |  113 +
 .../authorizer/SimpleWhitelistAuthorizer.java   |   70 +
 .../auth/digest/DigestSaslTransportPlugin.java  |    1 +
 .../storm/security/auth/kerberos/AutoTGT.java   |  267 ++
 .../auth/kerberos/AutoTGTKrb5LoginModule.java   |  108 +
 .../kerberos/AutoTGTKrb5LoginModuleTest.java    |   44 +
 .../auth/kerberos/ClientCallbackHandler.java    |  104 +
 .../kerberos/KerberosSaslTransportPlugin.java   |  206 ++
 .../auth/kerberos/ServerCallbackHandler.java    |   86 +
 .../auth/kerberos/jaas_kerberos_cluster.conf    |   31 +
 .../auth/kerberos/jaas_kerberos_launcher.conf   |   12 +
 .../testing/SingleUserSimpleTransport.java      |   37 +
 .../state/TestTransactionalState.java           |   47 +
 .../transactional/state/TransactionalState.java |   56 +-
 .../storm/ui/InvalidRequestException.java       |   20 +
 .../jvm/backtype/storm/utils/DRPCClient.java    |   63 +-
 .../jvm/backtype/storm/utils/LocalState.java    |   15 +-
 .../jvm/backtype/storm/utils/NimbusClient.java  |   11 +-
 .../src/jvm/backtype/storm/utils/TestUtils.java |   34 +
 .../src/jvm/backtype/storm/utils/Utils.java     |  126 +-
 .../backtype/storm/utils/ZookeeperAuthInfo.java |    9 +-
 .../storm/utils/ZookeeperServerCnxnFactory.java |   84 +
 .../trident/drpc/ReturnResultsReducer.java      |   13 +-
 .../topology/state/TestTransactionalState.java  |   47 +
 .../topology/state/TransactionalState.java      |   58 +-
 .../src/native/worker-launcher/.autom4te.cfg    |   42 +
 .../worker-launcher/.deps/worker-launcher.Po    |    1 +
 .../src/native/worker-launcher/Makefile.am      |   32 +
 .../src/native/worker-launcher/configure.ac     |   50 +
 .../native/worker-launcher/impl/configuration.c |  340 ++
 .../native/worker-launcher/impl/configuration.h |   45 +
 .../src/native/worker-launcher/impl/main.c      |  210 ++
 .../worker-launcher/impl/worker-launcher.c      |  779 +++++
 .../worker-launcher/impl/worker-launcher.h      |  129 +
 .../worker-launcher/test/test-worker-launcher.c |  340 ++
 storm-core/src/py/storm/DistributedRPC.py       |   21 +-
 .../src/py/storm/DistributedRPCInvocations.py   |   80 +-
 storm-core/src/py/storm/Nimbus-remote           |    7 +
 storm-core/src/py/storm/Nimbus.py               |  636 +++-
 storm-core/src/py/storm/ttypes.py               | 1227 ++++++-
 storm-core/src/storm.thrift                     |   58 +-
 storm-core/src/ui/public/css/style.css          |    6 +
 .../test/clj/backtype/storm/cluster_test.clj    |   91 +-
 .../test/clj/backtype/storm/config_test.clj     |   11 +
 .../test/clj/backtype/storm/logviewer_test.clj  |  187 ++
 .../storm/messaging/netty_integration_test.clj  |    3 +-
 .../test/clj/backtype/storm/nimbus_test.clj     |  340 +-
 .../scheduler/multitenant_scheduler_test.clj    |  737 +++++
 .../storm/security/auth/AuthUtils_test.clj      |   16 +-
 .../auth/DefaultHttpCredentialsPlugin_test.clj  |   40 +
 .../storm/security/auth/ThriftClient_test.clj   |   28 +-
 .../storm/security/auth/ThriftServer_test.clj   |    8 +-
 .../backtype/storm/security/auth/auth_test.clj  |  363 ++-
 .../authorizer/DRPCSimpleAclAuthorizer_test.clj |  226 ++
 .../security/auth/auto_login_module_test.clj    |   91 +
 .../storm/security/auth/drpc-auth-alice.jaas    |    5 +
 .../storm/security/auth/drpc-auth-bob.jaas      |    5 +
 .../storm/security/auth/drpc-auth-charlie.jaas  |    5 +
 .../storm/security/auth/drpc-auth-server.jaas   |    6 +
 .../storm/security/auth/drpc_auth_test.clj      |  315 ++
 .../storm/security/auth/nimbus_auth_test.clj    |  181 ++
 .../test/clj/backtype/storm/submitter_test.clj  |   75 +
 .../test/clj/backtype/storm/supervisor_test.clj |  248 +-
 .../clj/backtype/storm/transactional_test.clj   |   27 +-
 storm-core/test/clj/backtype/storm/ui_test.clj  |   49 +
 .../utils/ZookeeperServerCnxnFactory_test.clj   |   35 +
 .../test/clj/backtype/storm/utils_test.clj      |   58 +-
 .../test/clj/storm/trident/state_test.clj       |   25 +-
 145 files changed, 18509 insertions(+), 1212 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index b2a37f9..f7448d9 100644
--- a/.gitignore
+++ b/.gitignore
@@ -27,4 +27,4 @@ target
 /.lein-plugins/
 *.ipr
 *.iws
-.idea
\ No newline at end of file
+.idea

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/SECURITY.md
----------------------------------------------------------------------
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 @@ IPsec to encrypt all traffic being sent between the hosts in the cluster.
 
 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)
 
-### DRPC
+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
 
-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.
+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
 
-### Supervisors
+```
+StormClient {
+   com.sun.security.auth.module.Krb5LoginModule required
+   doNotPrompt=false
+   useTicketCache=true
+   serviceName="$nimbus_user";
+};
+```
 
-Supervisors are only clients they are not servers, and as such don't need special restrictions.
+The end user can override this if they have a headless user that has a keytab.
 
-### Workers
+### Authorization Setup
 
-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.
+*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
 
-### 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.
 
   

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/bin/storm
----------------------------------------------------------------------
diff --git a/bin/storm b/bin/storm
index c9885a2..d657aa7 100755
--- a/bin/storm
+++ b/bin/storm
@@ -190,6 +190,18 @@ def kill(*args):
         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 @@ COMMANDS = {"jar": jar, "kill": kill, "shell": shell, "nimbus": nimbus, "ui": ui
             "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

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index bb1cb04..d482065 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -32,14 +32,25 @@ storm.zookeeper.connection.timeout: 15000
 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.task.launch.secs: 120
 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 @@ supervisor.slots.ports:
     - 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,13 +119,18 @@ supervisor.monitor.frequency.secs: 3
 #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
 
 task.heartbeat.frequency.secs: 3
 task.refresh.poll.secs: 10
+task.credentials.poll.secs: 30
 
 zmq.threads: 1
 zmq.linger.millis: 5000
@@ -140,5 +175,6 @@ topology.max.error.report.per.interval: 5
 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"

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/conf/jaas_digest.conf
----------------------------------------------------------------------
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"

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/conf/jaas_kerberos.conf
----------------------------------------------------------------------
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";
+};
+

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 2c0cbc1..8f1fe73 100644
--- a/pom.xml
+++ b/pom.xml
@@ -179,6 +179,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>
 
@@ -415,6 +418,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>

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/pom.xml
----------------------------------------------------------------------
diff --git a/storm-core/pom.xml b/storm-core/pom.xml
index fec6218..c6022fd 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>
@@ -171,11 +191,6 @@
             <artifactId>mockito-all</artifactId>
             <scope>test</scope>
         </dependency>
-        <dependency>
-            <groupId>org.clojars.runa</groupId>
-            <artifactId>conjure</artifactId>
-            <scope>test</scope>
-        </dependency>
     </dependencies>
     <build>
         <sourceDirectory>src/jvm</sourceDirectory>
@@ -238,7 +253,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>
@@ -303,7 +319,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>

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/clj/backtype/storm/LocalCluster.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/LocalCluster.clj b/storm-core/src/clj/backtype/storm/LocalCluster.clj
index 77f3b3f..db6e60c 100644
--- a/storm-core/src/clj/backtype/storm/LocalCluster.clj
+++ b/storm-core/src/clj/backtype/storm/LocalCluster.clj
@@ -43,6 +43,9 @@
                       topology
                       submit-opts))
 
+(defn -uploadNewCredentials [this name creds]
+  (.uploadNewCredentials (:nimbus (. this state)) name creds))
+
 (defn -shutdown [this]
   (kill-local-storm-cluster (. this state)))
 

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/clj/backtype/storm/LocalDRPC.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/LocalDRPC.clj b/storm-core/src/clj/backtype/storm/LocalDRPC.clj
index a6dab95..d4884e4 100644
--- a/storm-core/src/clj/backtype/storm/LocalDRPC.clj
+++ b/storm-core/src/clj/backtype/storm/LocalDRPC.clj
@@ -15,7 +15,7 @@
 ;; limitations under the License.
 (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
@@ -24,7 +24,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}]

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/clj/backtype/storm/bootstrap.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/bootstrap.clj b/storm-core/src/clj/backtype/storm/bootstrap.clj
index 5f34ff1..0b41cdd 100644
--- a/storm-core/src/clj/backtype/storm/bootstrap.clj
+++ b/storm-core/src/clj/backtype/storm/bootstrap.clj
@@ -23,10 +23,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
@@ -52,7 +53,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]))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/clj/backtype/storm/cluster.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/cluster.clj b/storm-core/src/clj/backtype/storm/cluster.clj
index b5c1e3b..7c43aea 100644
--- a/storm-core/src/clj/backtype/storm/cluster.clj
+++ b/storm-core/src/clj/backtype/storm/cluster.clj
@@ -14,38 +14,45 @@
 ;; See the License for the specific language governing permissions and
 ;; 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])
-  (set-data [this path data])  ;; if node does not exist, create persistent with this data 
+  (create-sequential [this path data acls])
+  (set-data [this path data acls])  ;; if node does not exist, create persistent with this data 
   (get-data [this path watch?])
   (get-children [this path watch?])
-  (mkdirs [this path])
+  (mkdirs [this path acls])
   (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
@@ -65,28 +72,28 @@
      (unregister [this id]
                  (swap! callbacks dissoc id))
 
-     (set-ephemeral-node [this path data]
-                         (zk/mkdirs zk (parent-path path))
+     (set-ephemeral-node [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 acls)
                                ))
-                           (zk/create-node zk path data :ephemeral)
+                           (zk/create-node zk path data :ephemeral acls)
                            ))
      
-     (create-sequential [this path data]
-       (zk/create-node zk path data :sequential))
+     (create-sequential [this path data acls]
+       (zk/create-node zk path data :sequential acls))
      
-     (set-data [this path data]
+     (set-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]
@@ -100,8 +107,8 @@
      (get-children [this path watch?]
                    (zk/get-children zk path watch?))
      
-     (mkdirs [this path]
-             (zk/mkdirs zk path))
+     (mkdirs [this path acls]
+             (zk/mkdirs zk path acls))
      
      (close [this]
             (reset! active false)
@@ -135,6 +142,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])
   )
@@ -146,12 +155,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]
   (str SUPERVISORS-SUBTREE "/" id))
@@ -174,6 +185,9 @@
 (defn error-path [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]
     (reset! cb-atom nil)
@@ -210,14 +224,15 @@
       (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]
@@ -228,12 +243,13 @@
                                              (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
      
@@ -288,14 +304,14 @@
         )
 
       (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]
         (delete-node cluster-state (workerbeat-path storm-id node port))
         )
 
       (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]
         (try-cause
@@ -312,11 +328,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]
@@ -326,7 +342,8 @@
           (set-data cluster-state (storm-path storm-id)
                                   (-> base
                                       (merge new-elems)
-                                      Utils/serialize))))
+                                      Utils/serialize)
+                                  acls)))
 
       (storm-base [this storm-id callback]
         (when callback
@@ -339,18 +356,29 @@
         )
 
       (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))
+               _ (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
@@ -360,7 +388,7 @@
 
       (errors [this storm-id component-id]
          (let [path (error-path storm-id component-id)
-               _ (mkdirs cluster-state path)
+               _ (mkdirs cluster-state path acls)
                children (get-children cluster-state path false)
                errors (dofor [c children]
                              (let [data (-> (get-data cluster-state (str path "/" c) false)

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/clj/backtype/storm/command/upload_credentials.clj
----------------------------------------------------------------------
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)))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/clj/backtype/storm/config.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/config.clj b/storm-core/src/clj/backtype/storm/config.clj
index 15be94d..dcdca63 100644
--- a/storm-core/src/clj/backtype/storm/config.clj
+++ b/storm-core/src/clj/backtype/storm/config.clj
@@ -14,12 +14,12 @@
 ;; See the License for the specific language governing permissions and
 ;; 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")
@@ -117,10 +117,13 @@
     (validate-configs-with-schemas conf)
     conf))
 
-(defn read-yaml-config [name]
-  (let [conf (clojurify-structure (Utils/findAndReadConfigFile name true))]
-    (validate-configs-with-schemas conf)
-    conf))
+(defn read-yaml-config
+  ([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]
   (let [ret (str (conf STORM-LOCAL-DIR) file-path-separator "nimbus")]
@@ -198,6 +201,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"))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/clj/backtype/storm/daemon/common.clj
----------------------------------------------------------------------
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
+  )) 
+

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/clj/backtype/storm/daemon/drpc.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/drpc.clj b/storm-core/src/clj/backtype/storm/daemon/drpc.clj
index df07343..a7d3c32 100644
--- a/storm-core/src/clj/backtype/storm/daemon/drpc.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/drpc.clj
@@ -14,17 +14,22 @@
 ;; See the License for the specific language governing permissions and
 ;; 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.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)
@@ -40,17 +45,32 @@
         ))
   (@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 {})
         id->start (atom {})
+        id->func (atom {})
         request-queues (atom {})
         cleanup (fn [id] (swap! id->sem dissoc id)
                          (swap! id->result dissoc id)
-                         (swap! id->start dissoc id))
+                         (swap! id->start dissoc id)
+                         (swap! id->func dissoc id))
         my-ip (.getHostAddress (InetAddress/getLocalHost))
         clear-thread (async-loop
                       (fn []
@@ -66,12 +86,16 @@
         ]
     (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)
               ^ConcurrentLinkedQueue queue (acquire-queue request-queues function)
               ]
+          (swap! id->func assoc id function)
           (swap! id->start assoc id (current-time-secs))
           (swap! id->sem assoc id sem)
           (.add queue req)
@@ -87,19 +111,30 @@
               ))))
       DistributedRPCInvocations$Iface
       (^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->func 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->func 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
@@ -112,35 +147,90 @@
         (.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)
+          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 (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))))
+          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!))


[03/14] STORM-216: Added Authentication and Authorization.

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/backtype/storm/cluster_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/cluster_test.clj b/storm-core/test/clj/backtype/storm/cluster_test.clj
index 987429b..ba94ab1 100644
--- a/storm-core/test/clj/backtype/storm/cluster_test.clj
+++ b/storm-core/test/clj/backtype/storm/cluster_test.clj
@@ -16,6 +16,14 @@
 (ns backtype.storm.cluster-test
   (:import [java.util Arrays])
   (:import [backtype.storm.daemon.common Assignment StormBase SupervisorInfo])
+  (:import [org.apache.zookeeper ZooDefs ZooDefs$Ids])
+  (:import [org.mockito Mockito])
+  (:import [org.mockito.exceptions.base MockitoAssertionError])
+  (:import [org.apache.curator.framework CuratorFramework CuratorFrameworkFactory CuratorFrameworkFactory$Builder])
+  (:import [backtype.storm.utils Utils TestUtils ZookeeperAuthInfo])
+  (:require [backtype.storm [zookeeper :as zk]])
+  (:require [conjure.core])
+  (:use [conjure core])
   (:use [clojure test])
   (:use [backtype.storm cluster config util testing]))
 
@@ -25,7 +33,8 @@
           STORM-ZOOKEEPER-SERVERS ["localhost"]}))
 
 (defn mk-state
-  ([zk-port] (mk-distributed-cluster-state (mk-config zk-port)))
+  ([zk-port] (let [conf (mk-config zk-port)]
+               (mk-distributed-cluster-state conf :auth-conf conf)))
   ([zk-port cb]
      (let [ret (mk-state zk-port)]
        (.register ret cb)
@@ -36,16 +45,16 @@
 (deftest test-basics
   (with-inprocess-zookeeper zk-port
     (let [state (mk-state zk-port)]
-      (.set-data state "/root" (barr 1 2 3))
+      (.set-data state "/root" (barr 1 2 3) ZooDefs$Ids/OPEN_ACL_UNSAFE)
       (is (Arrays/equals (barr 1 2 3) (.get-data state "/root" false)))
       (is (= nil (.get-data state "/a" false)))
-      (.set-data state "/root/a" (barr 1 2))
-      (.set-data state "/root" (barr 1))
+      (.set-data state "/root/a" (barr 1 2) ZooDefs$Ids/OPEN_ACL_UNSAFE)
+      (.set-data state "/root" (barr 1) ZooDefs$Ids/OPEN_ACL_UNSAFE)
       (is (Arrays/equals (barr 1) (.get-data state "/root" false)))
       (is (Arrays/equals (barr 1 2) (.get-data state "/root/a" false)))
-      (.set-data state "/a/b/c/d" (barr 99))
+      (.set-data state "/a/b/c/d" (barr 99) ZooDefs$Ids/OPEN_ACL_UNSAFE)
       (is (Arrays/equals (barr 99) (.get-data state "/a/b/c/d" false)))
-      (.mkdirs state "/lalala")
+      (.mkdirs state "/lalala" ZooDefs$Ids/OPEN_ACL_UNSAFE)
       (is (= [] (.get-children state "/lalala" false)))
       (is (= #{"root" "a" "lalala"} (set (.get-children state "/" false))))
       (.delete-node state "/a")
@@ -58,7 +67,7 @@
   (with-inprocess-zookeeper zk-port
     (let [state1 (mk-state zk-port)
           state2 (mk-state zk-port)]
-      (.set-data state1 "/root" (barr 1))
+      (.set-data state1 "/root" (barr 1) ZooDefs$Ids/OPEN_ACL_UNSAFE)
       (is (Arrays/equals (barr 1) (.get-data state1 "/root" false)))
       (is (Arrays/equals (barr 1) (.get-data state2 "/root" false)))
       (.delete-node state2 "/root")
@@ -73,7 +82,7 @@
     (let [state1 (mk-state zk-port)
           state2 (mk-state zk-port)
           state3 (mk-state zk-port)]
-      (.set-ephemeral-node state1 "/a" (barr 1))
+      (.set-ephemeral-node state1 "/a" (barr 1) ZooDefs$Ids/OPEN_ACL_UNSAFE)
       (is (Arrays/equals (barr 1) (.get-data state1 "/a" false)))
       (is (Arrays/equals (barr 1) (.get-data state2 "/a" false)))
       (.close state3)
@@ -111,37 +120,37 @@
           state1 (mk-state zk-port state1-cb)
           [state2-last-cb state2-cb] (mk-callback-tester)
           state2 (mk-state zk-port state2-cb)]
-      (.set-data state1 "/root" (barr 1))
+      (.set-data state1 "/root" (barr 1) ZooDefs$Ids/OPEN_ACL_UNSAFE)
       (.get-data state2 "/root" true)
       (is (= nil @state1-last-cb))
       (is (= nil @state2-last-cb))
-      (.set-data state2 "/root" (barr 2))
+      (.set-data state2 "/root" (barr 2) ZooDefs$Ids/OPEN_ACL_UNSAFE)
       (is (= {:type :node-data-changed :path "/root"} (read-and-reset! state2-last-cb)))
       (is (= nil @state1-last-cb))
 
-      (.set-data state2 "/root" (barr 3))
+      (.set-data state2 "/root" (barr 3) ZooDefs$Ids/OPEN_ACL_UNSAFE)
       (is (= nil @state2-last-cb))
       (.get-data state2 "/root" true)
       (.get-data state2 "/root" false)
       (.delete-node state1 "/root")
       (is (= {:type :node-deleted :path "/root"} (read-and-reset! state2-last-cb)))
       (.get-data state2 "/root" true)
-      (.set-ephemeral-node state1 "/root" (barr 1 2 3 4))
+      (.set-ephemeral-node state1 "/root" (barr 1 2 3 4) ZooDefs$Ids/OPEN_ACL_UNSAFE)
       (is (= {:type :node-created :path "/root"} (read-and-reset! state2-last-cb)))
 
       (.get-children state1 "/" true)
-      (.set-data state2 "/a" (barr 9))
+      (.set-data state2 "/a" (barr 9) ZooDefs$Ids/OPEN_ACL_UNSAFE)
       (is (= nil @state2-last-cb))
       (is (= {:type :node-children-changed :path "/"} (read-and-reset! state1-last-cb)))
 
       (.get-data state2 "/root" true)
-      (.set-ephemeral-node state1 "/root" (barr 1 2))
+      (.set-ephemeral-node state1 "/root" (barr 1 2) ZooDefs$Ids/OPEN_ACL_UNSAFE)
       (is (= {:type :node-data-changed :path "/root"} (read-and-reset! state2-last-cb)))
 
-      (.mkdirs state1 "/ccc")
+      (.mkdirs state1 "/ccc" ZooDefs$Ids/OPEN_ACL_UNSAFE)
       (.get-children state1 "/ccc" true)
       (.get-data state2 "/ccc/b" true)
-      (.set-data state2 "/ccc/b" (barr 8))
+      (.set-data state2 "/ccc/b" (barr 8) ZooDefs$Ids/OPEN_ACL_UNSAFE)
       (is (= {:type :node-created :path "/ccc/b"} (read-and-reset! state2-last-cb)))
       (is (= {:type :node-children-changed :path "/ccc"} (read-and-reset! state1-last-cb)))
 
@@ -150,7 +159,7 @@
       (.close state1)
 
       (is (= {:type :node-deleted :path "/root"} (read-and-reset! state2-last-cb)))
-      (.set-data state2 "/root2" (barr 9))
+      (.set-data state2 "/root2" (barr 9) ZooDefs$Ids/OPEN_ACL_UNSAFE)
       (is (= {:type :node-created :path "/root2"} (read-and-reset! state2-last-cb)))
       (.close state2)
       )))
@@ -161,8 +170,8 @@
     (let [state (mk-storm-state zk-port)
           assignment1 (Assignment. "/aaa" {} {1 [2 2002 1]} {})
           assignment2 (Assignment. "/aaa" {} {1 [2 2002]} {})
-          base1 (StormBase. "/tmp/storm1" 1 {:type :active} 2 {})
-          base2 (StormBase. "/tmp/storm2" 2 {:type :active} 2 {})]
+          base1 (StormBase. "/tmp/storm1" 1 {:type :active} 2 {} "")
+          base2 (StormBase. "/tmp/storm2" 2 {:type :active} 2 {} "")]
       (is (= [] (.assignments state nil)))
       (.set-assignment! state "storm1" assignment1)
       (is (= assignment1 (.assignment-info state "storm1" nil)))
@@ -186,6 +195,11 @@
       (is (= base2 (.storm-base state "storm2" nil)))
       (is (= #{"storm2"} (set (.active-storms state))))
 
+      (is (nil? (.credentials state "storm1" nil)))
+      (.set-credentials! state "storm1" {"a" "a"} {})
+      (is (= {"a" "a"} (.credentials state "storm1" nil)))
+      (.set-credentials! state "storm1" {"b" "b"} {})
+      (is (= {"b" "b"} (.credentials state "storm1" nil)))
 
       ;; TODO add tests for task info and task heartbeat setting and getting
       (.disconnect state)
@@ -240,9 +254,44 @@
       (.disconnect state1)
       )))
 
+
+
+(deftest test-cluster-authentication
+  (with-inprocess-zookeeper zk-port
+    (let [builder (Mockito/mock CuratorFrameworkFactory$Builder)
+          conf (merge
+                (mk-config zk-port)
+                {STORM-ZOOKEEPER-CONNECTION-TIMEOUT 10
+                 STORM-ZOOKEEPER-SESSION-TIMEOUT 10
+                 STORM-ZOOKEEPER-RETRY-INTERVAL 5
+                 STORM-ZOOKEEPER-RETRY-TIMES 2
+                 STORM-ZOOKEEPER-RETRY-INTERVAL-CEILING 15
+                 STORM-ZOOKEEPER-AUTH-SCHEME "digest"
+                 STORM-ZOOKEEPER-AUTH-PAYLOAD "storm:thisisapoorpassword"})]
+      (. (Mockito/when (.connectString builder (Mockito/anyString))) (thenReturn builder))
+      (. (Mockito/when (.connectionTimeoutMs builder (Mockito/anyInt))) (thenReturn builder))
+      (. (Mockito/when (.sessionTimeoutMs builder (Mockito/anyInt))) (thenReturn builder))
+      (TestUtils/testSetupBuilder builder (str zk-port "/") conf (ZookeeperAuthInfo. conf))
+      (is (nil?
+           (try
+             (. (Mockito/verify builder) (authorization "digest" (.getBytes (conf STORM-ZOOKEEPER-AUTH-PAYLOAD))))
+             (catch MockitoAssertionError e
+               e)))))))
+
 (deftest test-storm-state-callbacks
   ;; TODO finish
   )
 
-
-
+(deftest test-cluster-state-default-acls
+  (testing "The default ACLs are empty."
+    (stubbing [zk/mkdirs nil
+               zk/mk-client (reify CuratorFramework (^void close [this] nil))]
+      (mk-distributed-cluster-state {})
+      (verify-call-times-for zk/mkdirs 1)
+      (verify-first-call-args-for-indices zk/mkdirs [2] nil))
+    (stubbing [mk-distributed-cluster-state nil
+               register nil
+               mkdirs nil]
+      (mk-storm-cluster-state {})
+      (verify-call-times-for mk-distributed-cluster-state 1)
+      (verify-first-call-args-for-indices mk-distributed-cluster-state [4] nil))))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/backtype/storm/config_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/config_test.clj b/storm-core/test/clj/backtype/storm/config_test.clj
index 01f788b..2b7f96e 100644
--- a/storm-core/test/clj/backtype/storm/config_test.clj
+++ b/storm-core/test/clj/backtype/storm/config_test.clj
@@ -84,6 +84,17 @@
     (is (thrown-cause? java.lang.IllegalArgumentException
       (.validateField validator "test" 42)))))
 
+(deftest test-positive-integer-validator
+  (let [validator ConfigValidation/PositiveIntegerValidator]
+    (doseq [x [42.42 -32 0 -0 "Forty-two"]]
+      (is (thrown-cause? java.lang.IllegalArgumentException
+        (.validateField validator "test" x))))
+
+    (doseq [x [42 4294967296 1 nil]]
+      (is (nil? (try
+                  (.validateField validator "test" x)
+                  (catch Exception e e)))))))
+
 (deftest test-worker-childopts-is-string-or-string-list
   (let [pass-cases [nil "some string" ["some" "string" "list"]]]
     (testing "worker.childopts validates"

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/backtype/storm/logviewer_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/logviewer_test.clj b/storm-core/test/clj/backtype/storm/logviewer_test.clj
new file mode 100644
index 0000000..37e63b9
--- /dev/null
+++ b/storm-core/test/clj/backtype/storm/logviewer_test.clj
@@ -0,0 +1,187 @@
+;; 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.logviewer-test
+  (:use [backtype.storm config util])
+  (:require [backtype.storm.daemon [logviewer :as logviewer]
+                                   [supervisor :as supervisor]])
+  (:require [conjure.core])
+  (:use [clojure test])
+  (:use [conjure core])
+  (:import [org.mockito Mockito]))
+
+(defmulti mk-mock-File #(:type %))
+
+(defmethod mk-mock-File :file [{file-name :name mtime :mtime
+                                :or {file-name "afile" mtime 1}}]
+  (let [mockFile (Mockito/mock java.io.File)]
+    (. (Mockito/when (.getName mockFile)) thenReturn file-name)
+    (. (Mockito/when (.lastModified mockFile)) thenReturn mtime)
+    (. (Mockito/when (.isFile mockFile)) thenReturn true)
+    (. (Mockito/when (.getCanonicalPath mockFile))
+       thenReturn (str "/mock/canonical/path/to/" file-name))
+    mockFile))
+
+(defmethod mk-mock-File :directory [{dir-name :name mtime :mtime
+                                     :or {dir-name "adir" mtime 1}}]
+  (let [mockDir (Mockito/mock java.io.File)]
+    (. (Mockito/when (.getName mockDir)) thenReturn dir-name)
+    (. (Mockito/when (.lastModified mockDir)) thenReturn mtime)
+    (. (Mockito/when (.isFile mockDir)) thenReturn false)
+    mockDir))
+
+(deftest test-mk-FileFilter-for-log-cleanup
+  (testing "log file filter selects the correct log files for purge"
+    (let [now-millis (current-time-millis)
+          conf {LOGVIEWER-CLEANUP-AGE-MINS 60
+                LOGVIEWER-CLEANUP-INTERVAL-SECS 300}
+          cutoff-millis (logviewer/cleanup-cutoff-age-millis conf now-millis)
+          old-mtime-millis (- cutoff-millis 500)
+          new-mtime-millis (+ cutoff-millis 500)
+          matching-files (map #(mk-mock-File %)
+                              [{:name "oldlog-1-2-worker-3.log"
+                                :type :file
+                                :mtime old-mtime-millis}
+                               {:name "oldlog-1-2-worker-3.log.8"
+                                :type :file
+                                :mtime old-mtime-millis}
+                               {:name "foobar*_topo-1-24242-worker-2834238.log"
+                                :type :file
+                                :mtime old-mtime-millis}])
+          excluded-files (map #(mk-mock-File %)
+                              [{:name "oldlog-1-2-worker-.log"
+                                :type :file
+                                :mtime old-mtime-millis}
+                               {:name "olddir-1-2-worker.log"
+                                :type :directory
+                                :mtime old-mtime-millis}
+                               {:name "newlog-1-2-worker.log"
+                                :type :file
+                                :mtime new-mtime-millis}
+                               {:name "some-old-file.txt"
+                                :type :file
+                                :mtime old-mtime-millis}
+                               {:name "metadata"
+                                :type :directory
+                                :mtime old-mtime-millis}
+                               {:name "newdir-1-2-worker.log"
+                                :type :directory
+                                :mtime new-mtime-millis}
+                               {:name "newdir"
+                                :type :directory
+                                :mtime new-mtime-millis}
+                              ])
+          file-filter (logviewer/mk-FileFilter-for-log-cleanup conf now-millis)]
+        (is   (every? #(.accept file-filter %) matching-files))
+        (is (not-any? #(.accept file-filter %) excluded-files))
+      )))
+
+(deftest test-get-log-root->files-map
+  (testing "returns map of root name to list of files"
+    (let [files (vec (map #(java.io.File. %) ["log-1-2-worker-3.log"
+                                              "log-1-2-worker-3.log.1"
+                                              "log-2-4-worker-6.log.1"]))
+          expected {"log-1-2-worker-3" #{(files 0) (files 1)}
+                    "log-2-4-worker-6" #{(files 2)}}]
+      (is (= expected (logviewer/get-log-root->files-map files))))))
+
+(deftest test-identify-worker-log-files
+  (testing "Does not include metadata file when there are any log files that
+           should not be cleaned up"
+    (let [cutoff-millis 2000
+          old-logFile (mk-mock-File {:name "mock-1-1-worker-1.log.1"
+                                     :type :file
+                                     :mtime (- cutoff-millis 1000)})
+          mock-metaFile (mk-mock-File {:name "mock-1-1-worker-1.yaml"
+                                       :type :file
+                                       :mtime 1})
+          new-logFile (mk-mock-File {:name "mock-1-1-worker-1.log"
+                                     :type :file
+                                     :mtime (+ cutoff-millis 1000)})
+          exp-id "id12345"
+          exp-user "alice"
+          expected {exp-id {:owner exp-user
+                            :files #{old-logFile}}}]
+      (stubbing [supervisor/read-worker-heartbeats nil
+                logviewer/get-metadata-file-for-log-root-name mock-metaFile
+                read-dir-contents [(.getName old-logFile) (.getName new-logFile)]
+                logviewer/get-worker-id-from-metadata-file exp-id
+                logviewer/get-topo-owner-from-metadata-file exp-user]
+        (is (= expected (logviewer/identify-worker-log-files [old-logFile] "/tmp/")))))))
+
+(deftest test-get-dead-worker-files-and-owners
+  (testing "removes any files of workers that are still alive"
+    (let [conf {SUPERVISOR-WORKER-TIMEOUT-SECS 5}
+          id->hb {"42" {:time-secs 1}}
+          now-secs 2
+          log-files #{:expected-file :unexpected-file}
+          exp-owner "alice"]
+      (stubbing [logviewer/identify-worker-log-files {"42" {:owner exp-owner
+                                                            :files #{:unexpected-file}}
+                                                      "007" {:owner exp-owner
+                                                             :files #{:expected-file}}}
+                 logviewer/get-topo-owner-from-metadata-file "alice"
+                 supervisor/read-worker-heartbeats id->hb]
+        (is (= [{:owner exp-owner :files #{:expected-file}}]
+               (logviewer/get-dead-worker-files-and-owners conf now-secs log-files "/tmp/")))))))
+
+(deftest test-cleanup-fn
+  (testing "cleanup function removes file as user when one is specified"
+    (let [exp-user "mock-user"
+          mockfile1 (mk-mock-File {:name "file1" :type :file})
+          mockfile2 (mk-mock-File {:name "file2" :type :file})
+          mockfile3 (mk-mock-File {:name "file3" :type :file})
+          mockyaml  (mk-mock-File {:name "foo.yaml" :type :file})
+          exp-cmd (str "rmr /mock/canonical/path/to/" (.getName mockfile3))]
+      (stubbing [logviewer/select-files-for-cleanup
+                   [(mk-mock-File {:name "throwaway" :type :file})]
+                 logviewer/get-dead-worker-files-and-owners
+                   [{:owner nil :files #{mockfile1}}
+                    {:files #{mockfile2}}
+                    {:owner exp-user :files #{mockfile3 mockyaml}}]
+                 supervisor/worker-launcher nil
+                 rmr nil]
+        (logviewer/cleanup-fn! "/tmp/")
+        (verify-call-times-for supervisor/worker-launcher 1)
+        (verify-first-call-args-for-indices supervisor/worker-launcher
+                                            [1 2] exp-user exp-cmd)
+        (verify-call-times-for rmr 3)
+        (verify-nth-call-args-for 1 rmr (.getCanonicalPath mockfile1))
+        (verify-nth-call-args-for 2 rmr (.getCanonicalPath mockfile2))
+        (verify-nth-call-args-for 3 rmr (.getCanonicalPath mockyaml))))))
+
+(deftest test-authorized-log-user
+  (testing "allow cluster admin"
+    (let [conf {NIMBUS-ADMINS ["alice"]}]
+      (stubbing [logviewer/get-log-user-whitelist []]
+        (is (logviewer/authorized-log-user? "alice" "non-blank-fname" conf)))))
+
+  (testing "ignore any cluster-set topology.users"
+    (let [conf {TOPOLOGY-USERS ["alice"]}]
+      (stubbing [logviewer/get-log-user-whitelist []]
+        (is (not (logviewer/authorized-log-user? "alice" "non-blank-fname" conf))))))
+
+  (testing "allow cluster logs user"
+    (let [conf {LOGS-USERS ["alice"]}]
+      (stubbing [logviewer/get-log-user-whitelist []]
+        (is (logviewer/authorized-log-user? "alice" "non-blank-fname" conf)))))
+
+  (testing "allow whitelisted topology user"
+    (stubbing [logviewer/get-log-user-whitelist ["alice"]]
+      (is (logviewer/authorized-log-user? "alice" "non-blank-fname" {}))))
+
+  (testing "disallow user not in nimbus admin, topo user, logs user, or whitelist"
+    (stubbing [logviewer/get-log-user-whitelist []]
+      (is (not (logviewer/authorized-log-user? "alice" "non-blank-fname" {}))))))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/backtype/storm/messaging/netty_integration_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/messaging/netty_integration_test.clj b/storm-core/test/clj/backtype/storm/messaging/netty_integration_test.clj
index 31e69e8..8534c82 100644
--- a/storm-core/test/clj/backtype/storm/messaging/netty_integration_test.clj
+++ b/storm-core/test/clj/backtype/storm/messaging/netty_integration_test.clj
@@ -30,8 +30,7 @@
                                                     STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000 
                                                     STORM-MESSAGING-NETTY-MAX-SLEEP-MS 5000
                                                     STORM-MESSAGING-NETTY-CLIENT-WORKER-THREADS 1
-                                                    STORM-MESSAGING-NETTY-SERVER-WORKER-THREADS 1
-                                                    }]
+                                                    STORM-MESSAGING-NETTY-SERVER-WORKER-THREADS 1}]
     (let [topology (thrift/mk-topology
                      {"1" (thrift/mk-spout-spec (TestWordSpout. true) :parallelism-hint 4)}
                      {"2" (thrift/mk-bolt-spec {"1" :shuffle} (TestGlobalCount.)

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/backtype/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/nimbus_test.clj b/storm-core/test/clj/backtype/storm/nimbus_test.clj
index 1e6554a..c902119 100644
--- a/storm-core/test/clj/backtype/storm/nimbus_test.clj
+++ b/storm-core/test/clj/backtype/storm/nimbus_test.clj
@@ -15,12 +15,15 @@
 ;; limitations under the License.
 (ns backtype.storm.nimbus-test
   (:use [clojure test])
+  (:require [backtype.storm [util :as util]])
   (:require [backtype.storm.daemon [nimbus :as nimbus]])
-  
   (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter])
   (:import [backtype.storm.scheduler INimbus])
+  (:import [backtype.storm.generated Credentials])
   (:use [backtype.storm bootstrap testing])
   (:use [backtype.storm.daemon common])
+  (:require [conjure.core])
+  (:use [conjure core])
   )
 
 (bootstrap)
@@ -159,6 +162,8 @@
       (is (not-nil? ((:executor->start-time-secs assignment) e))))
     ))
 
+ 	
+
 (deftest test-bogusId
   (with-local-cluster [cluster :supervisors 4 :ports-per-supervisor 3 :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0}]
     (let [state (:storm-cluster-state cluster)
@@ -167,6 +172,7 @@
        (is (thrown? NotAliveException (.getTopology nimbus "bogus-id")))
        (is (thrown? NotAliveException (.getUserTopology nimbus "bogus-id")))
        (is (thrown? NotAliveException (.getTopologyInfo nimbus "bogus-id")))
+       (is (thrown? NotAliveException (.uploadNewCredentials nimbus "bogus-id" (Credentials.))))
       )))
 
 (deftest test-assignment
@@ -724,10 +730,77 @@
       
       )))
 
+
+(defn check-for-collisions [state]
+ (log-message "Checking for collision")
+ (let [assignments (.assignments state nil)]
+   (log-message "Assignemts: " assignments)
+   (let [id->node->ports (into {} (for [id assignments
+                                                :let [executor->node+port (:executor->node+port (.assignment-info state id nil))
+                                                      node+ports (set (.values executor->node+port))
+                                                      node->ports (apply merge-with (fn [a b] (distinct (concat a b))) (for [[node port] node+ports] {node [port]}))]]
+                                                {id node->ports}))
+         _ (log-message "id->node->ports: " id->node->ports)
+         all-nodes (apply merge-with (fn [a b] 
+                                        (let [ret (concat a b)]
+                                              (log-message "Can we combine " (pr-str a) " and " (pr-str b) " without collisions? " (apply distinct? ret) " => " (pr-str ret)) 
+                                              (is (apply distinct? ret))
+                                              (distinct ret)))
+                          (.values id->node->ports))]
+)))
+
+(deftest test-rebalance-constrained-cluster
+  (with-simulated-time-local-cluster [cluster :supervisors 1 :ports-per-supervisor 4
+    :daemon-conf {SUPERVISOR-ENABLE false
+                  NIMBUS-MONITOR-FREQ-SECS 10
+                  TOPOLOGY-MESSAGE-TIMEOUT-SECS 30
+                  TOPOLOGY-ACKER-EXECUTORS 0}]
+    (letlocals
+      (bind topology (thrift/mk-topology
+                        {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
+                        {}))
+      (bind topology2 (thrift/mk-topology
+                        {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
+                        {}))
+      (bind topology3 (thrift/mk-topology
+                        {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
+                        {}))
+      (bind state (:storm-cluster-state cluster))
+      (submit-local-topology (:nimbus cluster)
+                             "test"
+                             {TOPOLOGY-WORKERS 3
+                              TOPOLOGY-MESSAGE-TIMEOUT-SECS 90} topology)
+      (submit-local-topology (:nimbus cluster)
+                             "test2"
+                             {TOPOLOGY-WORKERS 3
+                              TOPOLOGY-MESSAGE-TIMEOUT-SECS 90} topology2)
+      (submit-local-topology (:nimbus cluster)
+                             "test3"
+                             {TOPOLOGY-WORKERS 3
+                              TOPOLOGY-MESSAGE-TIMEOUT-SECS 90} topology3)
+
+      (advance-cluster-time cluster 31)
+
+      (check-for-collisions state)
+      (.rebalance (:nimbus cluster) "test" (doto (RebalanceOptions.)
+                    (.set_num_workers 4)
+                    (.set_wait_secs 0)
+                    ))
+
+      (advance-cluster-time cluster 11)
+      (check-for-collisions state)
+
+      (advance-cluster-time cluster 30)
+      (check-for-collisions state)
+      )))
+
+
 (deftest test-submit-invalid
   (with-simulated-time-local-cluster [cluster
     :daemon-conf {SUPERVISOR-ENABLE false
-                  TOPOLOGY-ACKER-EXECUTORS 0}]
+                  TOPOLOGY-ACKER-EXECUTORS 0
+                  NIMBUS-EXECUTORS-PER-TOPOLOGY 8
+                  NIMBUS-SLOTS-PER-TOPOLOGY 8}]
     (letlocals
       (bind topology (thrift/mk-topology
                         {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 0 :conf {TOPOLOGY-TASKS 1})}
@@ -747,7 +820,31 @@
                                "test/aaa"
                                {}
                                topology)))
-      )))
+      (bind topology (thrift/mk-topology
+                      {"1" (thrift/mk-spout-spec (TestPlannerSpout. true)
+                                                 :parallelism-hint 16
+                                                 :conf {TOPOLOGY-TASKS 16})}
+                      {}))
+      (bind state (:storm-cluster-state cluster))
+      (is (thrown? InvalidTopologyException
+                   (submit-local-topology (:nimbus cluster)
+                                          "test"
+                                          {TOPOLOGY-WORKERS 3} 
+                                          topology)))
+      (bind topology (thrift/mk-topology
+                      {"1" (thrift/mk-spout-spec (TestPlannerSpout. true)
+                                                 :parallelism-hint 5
+                                                 :conf {TOPOLOGY-TASKS 5})}
+                      {}))
+      (is (thrown? InvalidTopologyException
+                   (submit-local-topology (:nimbus cluster)
+                                          "test"
+                                          {TOPOLOGY-WORKERS 16}
+                                          topology)))
+      (is (nil? (submit-local-topology (:nimbus cluster)
+                                       "test"
+                                       {TOPOLOGY-WORKERS 8}
+                                       topology))))))
 
 (deftest test-cleans-corrupt
   (with-inprocess-zookeeper zk-port
@@ -775,13 +872,13 @@
        (.disconnect cluster-state)
        ))))
 
-(deftest test-no-overlapping-slots
-  ;; test that same node+port never appears across 2 assignments
-  )
+;(deftest test-no-overlapping-slots
+;  ;; test that same node+port never appears across 2 assignments
+;  )
 
-(deftest test-stateless
-  ;; test that nimbus can die and restart without any problems
-  )
+;(deftest test-stateless
+;  ;; test that nimbus can die and restart without any problems
+;  )
 
 (deftest test-clean-inbox
   "Tests that the inbox correctly cleans jar files."
@@ -816,11 +913,228 @@
        (assert-files-in-dir [])
        ))))
 
+(deftest test-nimbus-iface-submitTopologyWithOpts-checks-authorization
+  (with-local-cluster [cluster 
+                       :daemon-conf {NIMBUS-AUTHORIZER 
+                          "backtype.storm.security.auth.authorizer.DenyAuthorizer"}]
+    (let [
+          nimbus (:nimbus cluster)
+          topology (thrift/mk-topology {} {})
+         ]
+      (is (thrown? AuthorizationException
+          (submit-local-topology-with-opts nimbus "mystorm" {} topology 
+            (SubmitOptions. TopologyInitialStatus/INACTIVE))
+        ))
+    )
+  )
+)
+
+(deftest test-nimbus-iface-methods-check-authorization
+  (with-local-cluster [cluster 
+                       :daemon-conf {NIMBUS-AUTHORIZER 
+                          "backtype.storm.security.auth.authorizer.DenyAuthorizer"}]
+    (let [
+          nimbus (:nimbus cluster)
+          topology (thrift/mk-topology {} {})
+         ]
+      ; Fake good authorization as part of setup.
+      (mocking [nimbus/check-authorization!]
+          (submit-local-topology-with-opts nimbus "test" {} topology 
+              (SubmitOptions. TopologyInitialStatus/INACTIVE))
+      )
+      (stubbing [nimbus/storm-active? true]
+        (is (thrown? AuthorizationException
+          (.rebalance nimbus "test" (RebalanceOptions.))
+          ))
+      )
+      (is (thrown? AuthorizationException
+        (.activate nimbus "test")
+        ))
+      (is (thrown? AuthorizationException
+        (.deactivate nimbus "test")
+        ))
+    )
+  )
+)
+
+(deftest test-nimbus-check-authorization-params
+  (with-local-cluster [cluster
+                       :daemon-conf {NIMBUS-AUTHORIZER "backtype.storm.security.auth.authorizer.NoopAuthorizer"}]
+    (let [nimbus (:nimbus cluster)
+          topology-name "test-nimbus-check-autho-params"
+          topology (thrift/mk-topology {} {})]
+
+      (submit-local-topology-with-opts nimbus topology-name {} topology
+          (SubmitOptions. TopologyInitialStatus/INACTIVE))
+
+      (let [expected-name topology-name
+            expected-conf {TOPOLOGY-NAME expected-name
+                           :foo :bar}]
+
+        (testing "getTopologyConf calls check-authorization! with the correct parameters."
+          (let [expected-operation "getTopologyConf"]
+            (stubbing [nimbus/check-authorization! nil
+                       nimbus/try-read-storm-conf expected-conf
+                       util/to-json nil]
+              (try
+                (.getTopologyConf nimbus "fake-id")
+                (catch NotAliveException e)
+                (finally
+                  (verify-first-call-args-for-indices
+                    nimbus/check-authorization!
+                      [1 2 3] expected-name expected-conf expected-operation)
+                  (verify-first-call-args-for util/to-json expected-conf))))))
+
+        (testing "getTopology calls check-authorization! with the correct parameters."
+          (let [expected-operation "getTopology"]
+            (stubbing [nimbus/check-authorization! nil
+                       nimbus/try-read-storm-conf expected-conf
+                       nimbus/try-read-storm-topology nil
+                       system-topology! nil]
+              (try
+                (.getTopology nimbus "fake-id")
+                (catch NotAliveException e)
+                (finally
+                  (verify-first-call-args-for-indices
+                    nimbus/check-authorization!
+                      [1 2 3] expected-name expected-conf expected-operation)
+                  (verify-first-call-args-for-indices
+                    system-topology! [0] expected-conf))))))
+
+        (testing "getUserTopology calls check-authorization with the correct parameters."
+          (let [expected-operation "getUserTopology"]
+            (stubbing [nimbus/check-authorization! nil
+                       nimbus/try-read-storm-conf expected-conf
+                       nimbus/try-read-storm-topology nil]
+              (try
+                (.getUserTopology nimbus "fake-id")
+                (catch NotAliveException e)
+                (finally
+                  (verify-first-call-args-for-indices
+                    nimbus/check-authorization!
+                      [1 2 3] expected-name expected-conf expected-operation)
+                  (verify-first-call-args-for-indices
+                    nimbus/try-read-storm-topology [0] expected-conf))))))))))
+
+(deftest test-nimbus-iface-getTopology-methods-throw-correctly
+  (with-local-cluster [cluster]
+    (let [
+          nimbus (:nimbus cluster)
+          id "bogus ID"
+         ]
+      (is (thrown? NotAliveException (.getTopology nimbus id)))
+      (try
+        (.getTopology nimbus id)
+        (catch NotAliveException e
+           (is (= id (.get_msg e)))
+        )
+      )
+
+      (is (thrown? NotAliveException (.getTopologyConf nimbus id)))
+      (try (.getTopologyConf nimbus id)
+        (catch NotAliveException e
+           (is (= id (.get_msg e)))
+        )
+      )
+
+      (is (thrown? NotAliveException (.getTopologyInfo nimbus id)))
+      (try (.getTopologyInfo nimbus id)
+        (catch NotAliveException e
+           (is (= id (.get_msg e)))
+        )
+      )
+
+      (is (thrown? NotAliveException (.getUserTopology nimbus id)))
+      (try (.getUserTopology nimbus id)
+        (catch NotAliveException e
+           (is (= id (.get_msg e)))
+        )
+      )
+    )
+  )
+)
+
+(deftest test-nimbus-iface-getClusterInfo-filters-topos-without-bases
+  (with-local-cluster [cluster]
+    (let [
+          nimbus (:nimbus cluster)
+          bogus-secs 42
+          bogus-type "bogusType"
+          bogus-bases {
+                 "1" nil
+                 "2" {:launch-time-secs bogus-secs
+                        :storm-name "id2-name"
+                        :status {:type bogus-type}}
+                 "3" nil
+                 "4" {:launch-time-secs bogus-secs
+                        :storm-name "id4-name"
+                        :status {:type bogus-type}}
+                }
+        ]
+      (stubbing [topology-bases bogus-bases]
+        (let [topos (.get_topologies (.getClusterInfo nimbus))]
+          ; The number of topologies in the summary is correct.
+          (is (= (count 
+            (filter (fn [b] (second b)) bogus-bases)) (count topos)))
+          ; Each topology present has a valid name.
+          (is (empty?
+            (filter (fn [t] (or (nil? t) (nil? (.get_name t)))) topos)))
+          ; The topologies are those with valid bases.
+          (is (empty?
+            (filter (fn [t] 
+              (or 
+                (nil? t) 
+                (not (number? (read-string (.get_id t))))
+                (odd? (read-string (.get_id t)))
+              )) topos)))
+        )
+      )
+    )
+  )
+)
+
+(deftest test-defserverfn-numbus-iface-instance
+  (test-nimbus-iface-submitTopologyWithOpts-checks-authorization)
+  (test-nimbus-iface-methods-check-authorization)
+  (test-nimbus-iface-getTopology-methods-throw-correctly)
+  (test-nimbus-iface-getClusterInfo-filters-topos-without-bases)
+)
+
+(deftest test-nimbus-data-acls
+  (testing "nimbus-data uses correct ACLs"
+    (let [scheme "digest"
+          digest "storm:thisisapoorpassword"
+          auth-conf {STORM-ZOOKEEPER-AUTH-SCHEME scheme
+                     STORM-ZOOKEEPER-AUTH-PAYLOAD digest}
+          expected-acls nimbus/NIMBUS-ZK-ACLS
+          fake-inimbus (reify INimbus (getForcedScheduler [this] nil))]
+      (stubbing [mk-authorization-handler nil
+                 cluster/mk-storm-cluster-state nil
+                 nimbus/file-cache-map nil
+                 uptime-computer nil
+                 new-instance nil
+                 mk-timer nil
+                 nimbus/mk-scheduler nil]
+        (nimbus/nimbus-data auth-conf fake-inimbus)
+        (verify-call-times-for cluster/mk-storm-cluster-state 1)
+        (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
+                                            expected-acls)))))
+
+(deftest test-file-bogus-download
+  (with-local-cluster [cluster :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0}]
+    (let [nimbus (:nimbus cluster)]
+      (is (thrown-cause? AuthorizationException (.beginFileDownload nimbus nil)))
+      (is (thrown-cause? AuthorizationException (.beginFileDownload nimbus "")))
+      (is (thrown-cause? AuthorizationException (.beginFileDownload nimbus "/bogus-path/foo")))
+      )))
+
 (deftest test-validate-topo-config-on-submit
   (with-local-cluster [cluster]
     (let [nimbus (:nimbus cluster)
           topology (thrift/mk-topology {} {})
-          bad-config {"topology.workers" "3"}]
-      (is (thrown-cause? InvalidTopologyException
-        (submit-local-topology-with-opts nimbus "test" bad-config topology
-                                         (SubmitOptions.)))))))
+          bad-config {"topology.isolate.machines" "2"}]
+      ; Fake good authorization as part of setup.
+      (mocking [nimbus/check-authorization!]
+        (is (thrown-cause? InvalidTopologyException
+          (submit-local-topology-with-opts nimbus "test" bad-config topology
+                                           (SubmitOptions.))))))))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/backtype/storm/scheduler/multitenant_scheduler_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/scheduler/multitenant_scheduler_test.clj b/storm-core/test/clj/backtype/storm/scheduler/multitenant_scheduler_test.clj
new file mode 100644
index 0000000..4e79240
--- /dev/null
+++ b/storm-core/test/clj/backtype/storm/scheduler/multitenant_scheduler_test.clj
@@ -0,0 +1,737 @@
+;; 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.scheduler.multitenant-scheduler-test
+  (:use [clojure test])
+  (:use [backtype.storm bootstrap config testing])
+  (:require [backtype.storm.daemon [nimbus :as nimbus]])
+  (:import [backtype.storm.generated StormTopology])
+  (:import [backtype.storm.scheduler Cluster SupervisorDetails WorkerSlot ExecutorDetails
+            SchedulerAssignmentImpl Topologies TopologyDetails])
+  (:import [backtype.storm.scheduler.multitenant Node NodePool FreePool DefaultPool
+            IsolatedPool MultitenantScheduler]))
+
+(bootstrap)
+
+(defn gen-supervisors [count]
+  (into {} (for [id (range count)
+                :let [supervisor (SupervisorDetails. (str "super" id) (str "host" id) (list ) (map int (list 1 2 3 4)))]]
+            {(.getId supervisor) supervisor})))
+
+(defn to-top-map [topologies]
+  (into {} (for [top topologies] {(.getId top) top})))
+
+(defn ed [id] (ExecutorDetails. (int id) (int id)))
+
+(defn mk-ed-map [arg]
+  (into {}
+    (for [[name start end] arg]
+      (into {}
+        (for [at (range start end)]
+          {(ed at) name})))))
+
+(deftest test-node
+  (let [supers (gen-supervisors 5)
+       cluster (Cluster. (nimbus/standalone-nimbus) supers {})
+       node-map (Node/getAllNodesFrom cluster)]
+    (is (= 5 (.size node-map)))
+    (let [node (.get node-map "super0")]
+      (is (= "super0" (.getId node)))
+      (is (= true (.isAlive node)))
+      (is (= 0 (.size (.getRunningTopologies node))))
+      (is (= true (.isTotallyFree node)))
+      (is (= 4 (.totalSlotsFree node)))
+      (is (= 0 (.totalSlotsUsed node)))
+      (is (= 4 (.totalSlots node)))
+      (.assign node "topology1" (list (ExecutorDetails. 1 1)) cluster)
+      (is (= 1 (.size (.getRunningTopologies node))))
+      (is (= false (.isTotallyFree node)))
+      (is (= 3 (.totalSlotsFree node)))
+      (is (= 1 (.totalSlotsUsed node)))
+      (is (= 4 (.totalSlots node)))
+      (.assign node "topology1" (list (ExecutorDetails. 2 2)) cluster)
+      (is (= 1 (.size (.getRunningTopologies node))))
+      (is (= false (.isTotallyFree node)))
+      (is (= 2 (.totalSlotsFree node)))
+      (is (= 2 (.totalSlotsUsed node)))
+      (is (= 4 (.totalSlots node)))
+      (.assign node "topology2" (list (ExecutorDetails. 1 1)) cluster)
+      (is (= 2 (.size (.getRunningTopologies node))))
+      (is (= false (.isTotallyFree node)))
+      (is (= 1 (.totalSlotsFree node)))
+      (is (= 3 (.totalSlotsUsed node)))
+      (is (= 4 (.totalSlots node)))
+      (.assign node "topology2" (list (ExecutorDetails. 2 2)) cluster)
+      (is (= 2 (.size (.getRunningTopologies node))))
+      (is (= false (.isTotallyFree node)))
+      (is (= 0 (.totalSlotsFree node)))
+      (is (= 4 (.totalSlotsUsed node)))
+      (is (= 4 (.totalSlots node)))
+      (.freeAllSlots node cluster)
+      (is (= 0 (.size (.getRunningTopologies node))))
+      (is (= true (.isTotallyFree node)))
+      (is (= 4 (.totalSlotsFree node)))
+      (is (= 0 (.totalSlotsUsed node)))
+      (is (= 4 (.totalSlots node)))
+    )))
+
+(deftest test-free-pool
+  (let [supers (gen-supervisors 5)
+       cluster (Cluster. (nimbus/standalone-nimbus) supers {})
+       node-map (Node/getAllNodesFrom cluster)
+       free-pool (FreePool. )]
+    ;; assign one node so it is not in the pool
+    (.assign (.get node-map "super0") "topology1" (list (ExecutorDetails. 1 1)) cluster)
+    (.init free-pool cluster node-map)
+    (is (= 4 (.nodesAvailable free-pool)))
+    (is (= (* 4 4) (.slotsAvailable free-pool)))
+    (let [ns-count-1 (.getNodeAndSlotCountIfSlotsWereTaken free-pool 1)
+          ns-count-3 (.getNodeAndSlotCountIfSlotsWereTaken free-pool 3)
+          ns-count-4 (.getNodeAndSlotCountIfSlotsWereTaken free-pool 4)
+          ns-count-5 (.getNodeAndSlotCountIfSlotsWereTaken free-pool 5)]
+      (is (= 1 (._nodes ns-count-1)))
+      (is (= 4 (._slots ns-count-1)))
+      (is (= 1 (._nodes ns-count-3)))
+      (is (= 4 (._slots ns-count-3)))
+      (is (= 1 (._nodes ns-count-4)))
+      (is (= 4 (._slots ns-count-4)))
+      (is (= 2 (._nodes ns-count-5)))
+      (is (= 8 (._slots ns-count-5)))
+    )
+    (let [nodes (.takeNodesBySlots free-pool 5)]
+      (is (= 2 (.size nodes)))
+      (is (= 8 (Node/countFreeSlotsAlive nodes)))
+      (is (= 8 (Node/countTotalSlotsAlive nodes)))
+      (is (= 2 (.nodesAvailable free-pool)))
+      (is (= (* 2 4) (.slotsAvailable free-pool)))
+    )
+    (let [nodes (.takeNodes free-pool 3)] ;;Only 2 should be left
+      (is (= 2 (.size nodes)))
+      (is (= 8 (Node/countFreeSlotsAlive nodes)))
+      (is (= 8 (Node/countTotalSlotsAlive nodes)))
+      (is (= 0 (.nodesAvailable free-pool)))
+      (is (= 0 (.slotsAvailable free-pool)))
+    )))
+
+(deftest test-default-pool-simple
+  (let [supers (gen-supervisors 5)
+       cluster (Cluster. (nimbus/standalone-nimbus) supers {})
+       node-map (Node/getAllNodesFrom cluster)
+       free-pool (FreePool. )
+       default-pool (DefaultPool. )
+       executor1 (ed 1)
+       executor2 (ed 2)
+       executor3 (ed 3)
+       topology1 (TopologyDetails. "topology1" 
+                   {TOPOLOGY-NAME "topology-name-1"}
+                   (StormTopology.)
+                   2
+                   {executor1 "spout1"
+                    executor2 "bolt1"
+                    executor3 "bolt2"})]
+    ;; assign one node so it is not in the pool
+    (.assign (.get node-map "super0") "topology1" (list executor1) cluster)
+    (.init free-pool cluster node-map)
+    (.init default-pool cluster node-map)
+    (is (= true (.canAdd default-pool topology1)))
+    (.addTopology default-pool topology1)
+    ;;Only 1 node is in the default-pool because only one nodes was scheduled already
+    (is (= 4 (.slotsAvailable default-pool)))
+    (is (= 1 (.nodesAvailable default-pool)))
+    (is (= (* 4 4) (.slotsAvailable free-pool)))
+    (is (= 4 (.nodesAvailable free-pool)))
+    (is (= 1 (.size (.getSlots (.getAssignmentById cluster "topology1")))))
+    (.scheduleAsNeeded default-pool (into-array NodePool [free-pool]))
+    (is (= 4 (.slotsAvailable default-pool)))
+    (is (= 1 (.nodesAvailable default-pool)))
+    (is (= (* 4 4) (.slotsAvailable free-pool)))
+    (is (= 4 (.nodesAvailable free-pool)))
+    (is (= 2 (.size (.getSlots (.getAssignmentById cluster "topology1")))))
+    (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))
+))
+
+(deftest test-default-pool-big-request
+  (let [supers (gen-supervisors 5)
+       cluster (Cluster. (nimbus/standalone-nimbus) supers {})
+       node-map (Node/getAllNodesFrom cluster)
+       free-pool (FreePool. )
+       default-pool (DefaultPool. )
+       executor1 (ed 1)
+       executor2 (ed 2)
+       executor3 (ed 3)
+       topology1 (TopologyDetails. "topology1" 
+                   {TOPOLOGY-NAME "topology-name-1"}
+                   (StormTopology.)
+                   5
+                   {executor1 "spout1"
+                    executor2 "bolt1"
+                    executor3 "bolt2"})]
+    ;; assign one node so it is not in the pool
+    (.assign (.get node-map "super0") "topology1" (list executor1) cluster)
+    (.init free-pool cluster node-map)
+    (.init default-pool cluster node-map)
+    (is (= true (.canAdd default-pool topology1)))
+    (.addTopology default-pool topology1)
+    ;;Only 1 node is in the default-pool because only one nodes was scheduled already
+    (is (= 4 (.slotsAvailable default-pool)))
+    (is (= 1 (.nodesAvailable default-pool)))
+    (is (= (* 4 4) (.slotsAvailable free-pool)))
+    (is (= 4 (.nodesAvailable free-pool)))
+    (is (= 1 (.size (.getSlots (.getAssignmentById cluster "topology1")))))
+    (.scheduleAsNeeded default-pool (into-array NodePool [free-pool]))
+    (is (= 4 (.slotsAvailable default-pool)))
+    (is (= 1 (.nodesAvailable default-pool)))
+    (is (= (* 4 4) (.slotsAvailable free-pool)))
+    (is (= 4 (.nodesAvailable free-pool)))
+    (is (= 3 (.size (.getSlots (.getAssignmentById cluster "topology1")))))
+    (is (= "Fully Scheduled (requested 5 slots, but could only use 3)" (.get (.getStatusMap cluster) "topology1")))
+))
+
+(deftest test-default-pool-big-request-2
+  (let [supers (gen-supervisors 1)
+       cluster (Cluster. (nimbus/standalone-nimbus) supers {})
+       node-map (Node/getAllNodesFrom cluster)
+       free-pool (FreePool. )
+       default-pool (DefaultPool. )
+       executor1 (ed 1)
+       executor2 (ed 2)
+       executor3 (ed 3)
+       executor4 (ed 4)
+       executor5 (ed 5)
+       topology1 (TopologyDetails. "topology1" 
+                   {TOPOLOGY-NAME "topology-name-1"}
+                   (StormTopology.)
+                   5
+                   {executor1 "spout1"
+                    executor2 "bolt1"
+                    executor3 "bolt1"
+                    executor4 "bolt1"
+                    executor5 "bolt2"})]
+    ;; assign one node so it is not in the pool
+    (.assign (.get node-map "super0") "topology1" (list executor1) cluster)
+    (.init free-pool cluster node-map)
+    (.init default-pool cluster node-map)
+    (is (= true (.canAdd default-pool topology1)))
+    (.addTopology default-pool topology1)
+    ;;Only 1 node is in the default-pool because only one nodes was scheduled already
+    (is (= 4 (.slotsAvailable default-pool)))
+    (is (= 1 (.nodesAvailable default-pool)))
+    (is (= 0 (.slotsAvailable free-pool)))
+    (is (= 0 (.nodesAvailable free-pool)))
+    (is (= 1 (.size (.getSlots (.getAssignmentById cluster "topology1")))))
+    (.scheduleAsNeeded default-pool (into-array NodePool [free-pool]))
+    (is (= 4 (.slotsAvailable default-pool)))
+    (is (= 1 (.nodesAvailable default-pool)))
+    (is (= 0 (.slotsAvailable free-pool)))
+    (is (= 0 (.nodesAvailable free-pool)))
+    (is (= 4 (.size (.getSlots (.getAssignmentById cluster "topology1")))))
+    (is (= "Running with fewer slots than requested (4/5)" (.get (.getStatusMap cluster) "topology1")))
+))
+
+(deftest test-default-pool-full
+  (let [supers (gen-supervisors 2) ;;make 2 supervisors but only schedule with one of them
+       single-super {(ffirst supers) (second (first supers))}
+       single-cluster (Cluster. (nimbus/standalone-nimbus) single-super {})
+       executor1 (ed 1)
+       executor2 (ed 2)
+       executor3 (ed 3)
+       executor4 (ed 4)
+       executor5 (ed 5)
+       topology1 (TopologyDetails. "topology1" 
+                   {TOPOLOGY-NAME "topology-name-1"}
+                   (StormTopology.)
+                   5
+                   {executor1 "spout1"
+                    executor2 "bolt1"
+                    executor3 "bolt2"
+                    executor4 "bolt3"
+                    executor5 "bolt4"})]
+    (let [node-map (Node/getAllNodesFrom single-cluster)
+         free-pool (FreePool. )
+         default-pool (DefaultPool. )]
+      (.init free-pool single-cluster node-map)
+      (.init default-pool single-cluster node-map)
+      (.addTopology default-pool topology1)
+      (.scheduleAsNeeded default-pool (into-array NodePool [free-pool]))
+      ;; The cluster should be full and have 4 slots used, but the topology would like 1 more
+      (is (= 4 (.size (.getUsedSlots single-cluster))))
+      (is (= "Running with fewer slots than requested (4/5)" (.get (.getStatusMap single-cluster) "topology1")))
+    )
+
+    (let [cluster (Cluster. (nimbus/standalone-nimbus) supers (.getAssignments single-cluster))
+         node-map (Node/getAllNodesFrom cluster)
+         free-pool (FreePool. )
+         default-pool (DefaultPool. )]
+      (.init free-pool cluster node-map)
+      (.init default-pool cluster node-map)
+      (.addTopology default-pool topology1)
+      (.scheduleAsNeeded default-pool (into-array NodePool [free-pool]))
+      ;; The cluster should now have 5 slots used
+      (is (= 5 (.size (.getUsedSlots cluster))))
+      (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))
+    )
+))
+
+
+(deftest test-default-pool-complex
+  (let [supers (gen-supervisors 5)
+       cluster (Cluster. (nimbus/standalone-nimbus) supers {})
+       node-map (Node/getAllNodesFrom cluster)
+       free-pool (FreePool. )
+       default-pool (DefaultPool. )
+       executor1 (ed 1)
+       executor2 (ed 2)
+       executor3 (ed 3)
+       executor11 (ed 11)
+       executor12 (ed 12)
+       executor13 (ed 13)
+       executor14 (ed 14)
+       topology1 (TopologyDetails. "topology1" 
+                   {TOPOLOGY-NAME "topology-name-1"}
+                   (StormTopology.)
+                   2
+                   {executor1 "spout1"
+                    executor2 "bolt1"
+                    executor3 "bolt2"})
+       topology2 (TopologyDetails. "topology2"
+                    {TOPOLOGY-NAME "topology-name-2"}
+                    (StormTopology.)
+                    4
+                    {executor11 "spout11"
+                     executor12 "bolt12"
+                     executor13 "bolt13"
+                     executor14 "bolt14"})]
+    ;; assign one node so it is not in the pool
+    (.assign (.get node-map "super0") "topology1" (list executor1) cluster)
+    (.init free-pool cluster node-map)
+    (.init default-pool cluster node-map)
+    (is (= true (.canAdd default-pool topology1)))
+    (.addTopology default-pool topology1)
+    (is (= true (.canAdd default-pool topology2)))
+    (.addTopology default-pool topology2)
+    ;;Only 1 node is in the default-pool because only one nodes was scheduled already
+    (is (= 4 (.slotsAvailable default-pool)))
+    (is (= 1 (.nodesAvailable default-pool)))
+    (is (= (* 4 4) (.slotsAvailable free-pool)))
+    (is (= 4 (.nodesAvailable free-pool)))
+    (is (= 1 (.size (.getSlots (.getAssignmentById cluster "topology1")))))
+    (is (= nil (.getAssignmentById cluster "topology2")))
+    (.scheduleAsNeeded default-pool (into-array NodePool [free-pool]))
+    ;;We steal a node from the free pool to handle the extra
+    (is (= 8 (.slotsAvailable default-pool)))
+    (is (= 2 (.nodesAvailable default-pool)))
+    (is (= (* 3 4) (.slotsAvailable free-pool)))
+    (is (= 3 (.nodesAvailable free-pool)))
+    (is (= 2 (.size (.getSlots (.getAssignmentById cluster "topology1")))))
+    (is (= 4 (.size (.getSlots (.getAssignmentById cluster "topology2")))))
+    (let [ns-count-1 (.getNodeAndSlotCountIfSlotsWereTaken default-pool 1)
+          ns-count-3 (.getNodeAndSlotCountIfSlotsWereTaken default-pool 3)
+          ns-count-4 (.getNodeAndSlotCountIfSlotsWereTaken default-pool 4)
+          ns-count-5 (.getNodeAndSlotCountIfSlotsWereTaken default-pool 5)]
+      (is (= 1 (._nodes ns-count-1)))
+      (is (= 4 (._slots ns-count-1)))
+      (is (= 1 (._nodes ns-count-3)))
+      (is (= 4 (._slots ns-count-3)))
+      (is (= 1 (._nodes ns-count-4)))
+      (is (= 4 (._slots ns-count-4)))
+      (is (= 2 (._nodes ns-count-5)))
+      (is (= 8 (._slots ns-count-5)))
+    )
+    (let [nodes (.takeNodesBySlots default-pool 3)]
+      (is (= 1 (.size nodes)))
+      (is (= 4 (Node/countFreeSlotsAlive nodes)))
+      (is (= 4 (Node/countTotalSlotsAlive nodes)))
+      (is (= 1 (.nodesAvailable default-pool)))
+      (is (= (* 1 4) (.slotsAvailable default-pool)))
+    )
+    (let [nodes (.takeNodes default-pool 3)] ;;Only 1 should be left
+      (is (= 1 (.size nodes)))
+      (is (= 4 (Node/countFreeSlotsAlive nodes)))
+      (is (= 4 (Node/countTotalSlotsAlive nodes)))
+      (is (= 0 (.nodesAvailable default-pool)))
+      (is (= 0 (.slotsAvailable default-pool)))
+    )
+    (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))
+    (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology2")))
+))
+
+(deftest test-isolated-pool-simple
+  (let [supers (gen-supervisors 5)
+       cluster (Cluster. (nimbus/standalone-nimbus) supers {})
+       node-map (Node/getAllNodesFrom cluster)
+       free-pool (FreePool. )
+       isolated-pool (IsolatedPool. 5)
+       executor1 (ed 1)
+       executor2 (ed 2)
+       executor3 (ed 3)
+       executor4 (ed 4)
+       topology1 (TopologyDetails. "topology1" 
+                   {TOPOLOGY-NAME "topology-name-1"
+                    TOPOLOGY-ISOLATED-MACHINES 4}
+                   (StormTopology.)
+                   4
+                   {executor1 "spout1"
+                    executor2 "bolt1"
+                    executor3 "bolt2"
+                    executor4 "bolt4"})]
+    ;; assign one node so it is not in the pool
+    (.assign (.get node-map "super0") "topology1" (list executor1) cluster)
+    (.init free-pool cluster node-map)
+    (.init isolated-pool cluster node-map)
+    (is (= true (.canAdd isolated-pool topology1)))
+    (.addTopology isolated-pool topology1)
+    ;;Isolated topologies cannot have their resources stolen
+    (is (= 0 (.slotsAvailable isolated-pool)))
+    (is (= 0 (.nodesAvailable isolated-pool)))
+    (is (= (* 4 4) (.slotsAvailable free-pool)))
+    (is (= 4 (.nodesAvailable free-pool)))
+    (is (= 1 (.size (.getSlots (.getAssignmentById cluster "topology1")))))
+    (.scheduleAsNeeded isolated-pool (into-array NodePool [free-pool]))
+    (is (= 0 (.slotsAvailable isolated-pool)))
+    (is (= 0 (.nodesAvailable isolated-pool)))
+    (is (= (* 1 4) (.slotsAvailable free-pool)))
+    (is (= 1 (.nodesAvailable free-pool)))
+    (let [assigned-slots (.getSlots (.getAssignmentById cluster "topology1"))]
+      ;; 4 slots on 4 machines
+      (is (= 4 (.size assigned-slots)))
+      (is (= 4 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot))))))
+    )
+    (is (= "Scheduled Isolated on 4 Nodes" (.get (.getStatusMap cluster) "topology1")))
+))
+
+(deftest test-isolated-pool-big-ask
+  (let [supers (gen-supervisors 5)
+       cluster (Cluster. (nimbus/standalone-nimbus) supers {})
+       node-map (Node/getAllNodesFrom cluster)
+       free-pool (FreePool. )
+       isolated-pool (IsolatedPool. 5)
+       executor1 (ed 1)
+       executor2 (ed 2)
+       executor3 (ed 3)
+       executor4 (ed 4)
+       topology1 (TopologyDetails. "topology1" 
+                   {TOPOLOGY-NAME "topology-name-1"
+                    TOPOLOGY-ISOLATED-MACHINES 4}
+                   (StormTopology.)
+                   10
+                   {executor1 "spout1"
+                    executor2 "bolt1"
+                    executor3 "bolt2"
+                    executor4 "bolt4"})]
+    ;; assign one node so it is not in the pool
+    (.assign (.get node-map "super0") "topology1" (list executor1) cluster)
+    (.init free-pool cluster node-map)
+    (.init isolated-pool cluster node-map)
+    (is (= true (.canAdd isolated-pool topology1)))
+    (.addTopology isolated-pool topology1)
+    ;;Isolated topologies cannot have their resources stolen
+    (is (= 0 (.slotsAvailable isolated-pool)))
+    (is (= 0 (.nodesAvailable isolated-pool)))
+    (is (= (* 4 4) (.slotsAvailable free-pool)))
+    (is (= 4 (.nodesAvailable free-pool)))
+    (is (= 1 (.size (.getSlots (.getAssignmentById cluster "topology1")))))
+    (.scheduleAsNeeded isolated-pool (into-array NodePool [free-pool]))
+    (is (= 0 (.slotsAvailable isolated-pool)))
+    (is (= 0 (.nodesAvailable isolated-pool)))
+    (is (= (* 1 4) (.slotsAvailable free-pool)))
+    (is (= 1 (.nodesAvailable free-pool)))
+    (let [assigned-slots (.getSlots (.getAssignmentById cluster "topology1"))]
+      ;; 4 slots on 4 machines
+      (is (= 4 (.size assigned-slots)))
+      (is (= 4 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot))))))
+    )
+    (is (= "Scheduled Isolated on 4 Nodes" (.get (.getStatusMap cluster) "topology1")))
+))
+
+(deftest test-isolated-pool-complex
+  (let [supers (gen-supervisors 5)
+       cluster (Cluster. (nimbus/standalone-nimbus) supers {})
+       node-map (Node/getAllNodesFrom cluster)
+       free-pool (FreePool. )
+       isolated-pool (IsolatedPool. 5)
+       executor1 (ed 1)
+       executor2 (ed 2)
+       executor3 (ed 3)
+       executor4 (ed 4)
+       executor11 (ed 11)
+       executor12 (ed 12)
+       executor13 (ed 13)
+       executor14 (ed 14)
+       topology1 (TopologyDetails. "topology1" 
+                   {TOPOLOGY-NAME "topology-name-1"}
+                   (StormTopology.)
+                   4
+                   {executor1 "spout1"
+                    executor2 "bolt1"
+                    executor3 "bolt2"
+                    executor4 "bolt4"})
+       topology2 (TopologyDetails. "topology2"
+                    {TOPOLOGY-NAME "topology-name-2"
+                     TOPOLOGY-ISOLATED-MACHINES 2}
+                    (StormTopology.)
+                    4
+                    {executor11 "spout11"
+                     executor12 "bolt12"
+                     executor13 "bolt13"
+                     executor14 "bolt14"})]
+    ;; assign one node so it is not in the pool
+    (.assign (.get node-map "super0") "topology1" (list executor1) cluster)
+    (.init free-pool cluster node-map)
+    (.init isolated-pool cluster node-map)
+    (is (= true (.canAdd isolated-pool topology1)))
+    (.addTopology isolated-pool topology1)
+    (is (= true (.canAdd isolated-pool topology2)))
+    (.addTopology isolated-pool topology2)
+    ;; nodes can be stolen from non-isolted tops in the pool
+    (is (= 4 (.slotsAvailable isolated-pool)))
+    (is (= 1 (.nodesAvailable isolated-pool)))
+    (is (= (* 4 4) (.slotsAvailable free-pool)))
+    (is (= 4 (.nodesAvailable free-pool)))
+    (is (= 1 (.size (.getSlots (.getAssignmentById cluster "topology1")))))
+    (is (= nil (.getAssignmentById cluster "topology2")))
+    (.scheduleAsNeeded isolated-pool (into-array NodePool [free-pool]))
+    ;;We steal 2 nodes from the free pool to handle the extra (but still only 1 node for the non-isolated top
+    (is (= 4 (.slotsAvailable isolated-pool)))
+    (is (= 1 (.nodesAvailable isolated-pool)))
+    (is (= (* 2 4) (.slotsAvailable free-pool)))
+    (is (= 2 (.nodesAvailable free-pool)))
+    (let [assigned-slots (.getSlots (.getAssignmentById cluster "topology1"))]
+      ;; 4 slots on 1 machine
+      (is (= 4 (.size assigned-slots)))
+      (is (= 1 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot))))))
+    )
+    (let [assigned-slots (.getSlots (.getAssignmentById cluster "topology2"))]
+      ;; 4 slots on 2 machines
+      (is (= 4 (.size assigned-slots)))
+      (is (= 2 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot))))))
+    )
+
+    (let [ns-count-1 (.getNodeAndSlotCountIfSlotsWereTaken isolated-pool 1)
+          ns-count-3 (.getNodeAndSlotCountIfSlotsWereTaken isolated-pool 3)
+          ns-count-4 (.getNodeAndSlotCountIfSlotsWereTaken isolated-pool 4)
+          ns-count-5 (.getNodeAndSlotCountIfSlotsWereTaken isolated-pool 5)]
+      (is (= 1 (._nodes ns-count-1)))
+      (is (= 4 (._slots ns-count-1)))
+      (is (= 1 (._nodes ns-count-3)))
+      (is (= 4 (._slots ns-count-3)))
+      (is (= 1 (._nodes ns-count-4)))
+      (is (= 4 (._slots ns-count-4)))
+      (is (= 1 (._nodes ns-count-5))) ;;Only 1 node can be stolen right now
+      (is (= 4 (._slots ns-count-5)))
+    )
+    (let [nodes (.takeNodesBySlots isolated-pool 3)]
+      (is (= 1 (.size nodes)))
+      (is (= 4 (Node/countFreeSlotsAlive nodes)))
+      (is (= 4 (Node/countTotalSlotsAlive nodes)))
+      (is (= 0 (.nodesAvailable isolated-pool)))
+      (is (= (* 0 4) (.slotsAvailable isolated-pool)))
+    )
+    (let [assigned-slots (.getSlots (.getAssignmentById cluster "topology1"))]
+      ;; 4 slots on 1 machine
+      (is (= 0 (.size assigned-slots)))
+      (is (= 0 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot))))))
+    )
+    (let [assigned-slots (.getSlots (.getAssignmentById cluster "topology2"))]
+      ;; 4 slots on 2 machines
+      (is (= 4 (.size assigned-slots)))
+      (is (= 2 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot))))))
+    )
+    (let [nodes (.takeNodes isolated-pool 3)] ;;Cannot steal from the isolated scheduler
+      (is (= 0 (.size nodes)))
+      (is (= 0 (Node/countFreeSlotsAlive nodes)))
+      (is (= 0 (Node/countTotalSlotsAlive nodes)))
+      (is (= 0 (.nodesAvailable isolated-pool)))
+      (is (= 0 (.slotsAvailable isolated-pool)))
+    )
+    (is (= "Scheduled Isolated on 1 Nodes" (.get (.getStatusMap cluster) "topology1")))
+    (is (= "Scheduled Isolated on 2 Nodes" (.get (.getStatusMap cluster) "topology2")))
+))
+
+(deftest test-isolated-pool-complex-2
+  (let [supers (gen-supervisors 5)
+       cluster (Cluster. (nimbus/standalone-nimbus) supers {})
+       node-map (Node/getAllNodesFrom cluster)
+       free-pool (FreePool. )
+       ;;like before but now we can only hold 2 nodes max.  Don't go over
+       isolated-pool (IsolatedPool. 2)
+       executor1 (ed 1)
+       executor2 (ed 2)
+       executor3 (ed 3)
+       executor4 (ed 4)
+       executor11 (ed 11)
+       executor12 (ed 12)
+       executor13 (ed 13)
+       executor14 (ed 14)
+       topology1 (TopologyDetails. "topology1" 
+                   {TOPOLOGY-NAME "topology-name-1"}
+                   (StormTopology.)
+                   4
+                   {executor1 "spout1"
+                    executor2 "bolt1"
+                    executor3 "bolt2"
+                    executor4 "bolt4"})
+       topology2 (TopologyDetails. "topology2"
+                    {TOPOLOGY-NAME "topology-name-2"
+                     TOPOLOGY-ISOLATED-MACHINES 2}
+                    (StormTopology.)
+                    4
+                    {executor11 "spout11"
+                     executor12 "bolt12"
+                     executor13 "bolt13"
+                     executor14 "bolt14"})]
+    ;; assign one node so it is not in the pool
+    (.assign (.get node-map "super0") "topology1" (list executor1) cluster)
+    (.init free-pool cluster node-map)
+    (.init isolated-pool cluster node-map)
+    (is (= true (.canAdd isolated-pool topology1)))
+    (.addTopology isolated-pool topology1)
+    (is (= true (.canAdd isolated-pool topology2)))
+    (.addTopology isolated-pool topology2)
+    ;; nodes can be stolen from non-isolted tops in the pool
+    (is (= 4 (.slotsAvailable isolated-pool)))
+    (is (= 1 (.nodesAvailable isolated-pool)))
+    (is (= (* 4 4) (.slotsAvailable free-pool)))
+    (is (= 4 (.nodesAvailable free-pool)))
+    (is (= 1 (.size (.getSlots (.getAssignmentById cluster "topology1")))))
+    (is (= nil (.getAssignmentById cluster "topology2")))
+    (.scheduleAsNeeded isolated-pool (into-array NodePool [free-pool]))
+    ;;We steal 1 node from the free pool and 1 from ourself to handle the extra
+    (is (= 0 (.slotsAvailable isolated-pool)))
+    (is (= 0 (.nodesAvailable isolated-pool)))
+    (is (= (* 3 4) (.slotsAvailable free-pool)))
+    (is (= 3 (.nodesAvailable free-pool)))
+    (let [assigned-slots (.getSlots (.getAssignmentById cluster "topology1"))]
+      ;; 0 slots on 0 machine
+      (is (= 0 (.size assigned-slots)))
+      (is (= 0 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot))))))
+    )
+    (let [assigned-slots (.getSlots (.getAssignmentById cluster "topology2"))]
+      ;; 4 slots on 2 machines
+      (is (= 4 (.size assigned-slots)))
+      (is (= 2 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot))))))
+    )
+    (is (= "Max Nodes(2) for this user would be exceeded. 1 more nodes needed to run topology." (.get (.getStatusMap cluster) "topology1")))
+    (is (= "Scheduled Isolated on 2 Nodes" (.get (.getStatusMap cluster) "topology2")))
+))
+
+(deftest test-multitenant-scheduler
+  (let [supers (gen-supervisors 10)
+       topology1 (TopologyDetails. "topology1" 
+                   {TOPOLOGY-NAME "topology-name-1"
+                    TOPOLOGY-SUBMITTER-USER "userC"}
+                   (StormTopology.)
+                   4
+                   (mk-ed-map [["spout1" 0 5]
+                               ["bolt1" 5 10]
+                               ["bolt2" 10 15]
+                               ["bolt3" 15 20]]))
+       topology2 (TopologyDetails. "topology2"
+                    {TOPOLOGY-NAME "topology-name-2"
+                     TOPOLOGY-ISOLATED-MACHINES 2
+                     TOPOLOGY-SUBMITTER-USER "userA"}
+                    (StormTopology.)
+                    4
+                    (mk-ed-map [["spout11" 0 5]
+                                ["bolt12" 5 6]
+                                ["bolt13" 6 7]
+                                ["bolt14" 7 10]]))
+       topology3 (TopologyDetails. "topology3"
+                    {TOPOLOGY-NAME "topology-name-3"
+                     TOPOLOGY-ISOLATED-MACHINES 5
+                     TOPOLOGY-SUBMITTER-USER "userB"}
+                    (StormTopology.)
+                    10
+                    (mk-ed-map [["spout21" 0 10]
+                                ["bolt22" 10 20]
+                                ["bolt23" 20 30]
+                                ["bolt24" 30 40]]))
+       cluster (Cluster. (nimbus/standalone-nimbus) supers {})
+       node-map (Node/getAllNodesFrom cluster)
+       topologies (Topologies. (to-top-map [topology1 topology2 topology3]))
+       conf {MULTITENANT-SCHEDULER-USER-POOLS {"userA" 5 "userB" 5}}
+       scheduler (MultitenantScheduler.)]
+    (.assign (.get node-map "super0") "topology1" (list (ed 1)) cluster)
+    (.assign (.get node-map "super1") "topology2" (list (ed 5)) cluster)
+    (.prepare scheduler conf)
+    (.schedule scheduler topologies cluster)
+    (let [assignment (.getAssignmentById cluster "topology1")
+          assigned-slots (.getSlots assignment)
+          executors (.getExecutors assignment)]
+      ;; 4 slots on 1 machine, all executors assigned
+      (is (= 4 (.size assigned-slots)))
+      (is (= 1 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot))))))
+      (is (= 20 (.size executors)))
+    )
+    (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))
+    (is (= "Scheduled Isolated on 2 Nodes" (.get (.getStatusMap cluster) "topology2")))
+    (is (= "Scheduled Isolated on 5 Nodes" (.get (.getStatusMap cluster) "topology3")))
+))
+
+
+(deftest test-multitenant-scheduler-bad-starting-state
+  (let [supers (gen-supervisors 10)
+       topology1 (TopologyDetails. "topology1" 
+                   {TOPOLOGY-NAME "topology-name-1"
+                    TOPOLOGY-SUBMITTER-USER "userC"}
+                   (StormTopology.)
+                   4
+                   (mk-ed-map [["spout1" 0 5]
+                               ["bolt1" 5 10]
+                               ["bolt2" 10 15]
+                               ["bolt3" 15 20]]))
+       topology2 (TopologyDetails. "topology2"
+                    {TOPOLOGY-NAME "topology-name-2"
+                     TOPOLOGY-ISOLATED-MACHINES 2
+                     TOPOLOGY-SUBMITTER-USER "userA"}
+                    (StormTopology.)
+                    4
+                    (mk-ed-map [["spout11" 0 5]
+                                ["bolt12" 5 6]
+                                ["bolt13" 6 7]
+                                ["bolt14" 7 10]]))
+       topology3 (TopologyDetails. "topology3"
+                    {TOPOLOGY-NAME "topology-name-3"
+                     TOPOLOGY-ISOLATED-MACHINES 5
+                     TOPOLOGY-SUBMITTER-USER "userB"}
+                    (StormTopology.)
+                    10
+                    (mk-ed-map [["spout21" 0 10]
+                                ["bolt22" 10 20]
+                                ["bolt23" 20 30]
+                                ["bolt24" 30 40]]))
+       existing-assignments {
+         "topology2" (SchedulerAssignmentImpl. "topology2" {(ExecutorDetails. 0 5) (WorkerSlot. "super1" 1)})
+         "topology3" (SchedulerAssignmentImpl. "topology3" {(ExecutorDetails. 0 10) (WorkerSlot. "super1" 1)})
+       }
+       cluster (Cluster. (nimbus/standalone-nimbus) supers existing-assignments)
+       topologies (Topologies. (to-top-map [topology1 topology2 topology3]))
+       conf {MULTITENANT-SCHEDULER-USER-POOLS {"userA" 5 "userB" 5}}
+       scheduler (MultitenantScheduler.)]
+    (.prepare scheduler conf)
+    (.schedule scheduler topologies cluster)
+    (let [assignment (.getAssignmentById cluster "topology1")
+          assigned-slots (.getSlots assignment)
+          executors (.getExecutors assignment)]
+      ;; 4 slots on 1 machine, all executors assigned
+      (is (= 4 (.size assigned-slots)))
+      (is (= 1 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot))))))
+      (is (= 20 (.size executors)))
+    )
+    (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))
+    (is (= "Scheduled Isolated on 2 Nodes" (.get (.getStatusMap cluster) "topology2")))
+    (is (= "Scheduled Isolated on 5 Nodes" (.get (.getStatusMap cluster) "topology3")))
+))
+
+

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/backtype/storm/security/auth/AuthUtils_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/security/auth/AuthUtils_test.clj b/storm-core/test/clj/backtype/storm/security/auth/AuthUtils_test.clj
index ed21904..2d96b18 100644
--- a/storm-core/test/clj/backtype/storm/security/auth/AuthUtils_test.clj
+++ b/storm-core/test/clj/backtype/storm/security/auth/AuthUtils_test.clj
@@ -14,13 +14,16 @@
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
 (ns backtype.storm.security.auth.AuthUtils-test
-  (:import [backtype.storm.security.auth AuthUtils])
+  (:import [backtype.storm.security.auth AuthUtils IAutoCredentials])
   (:import [java.io IOException])
   (:import [javax.security.auth.login AppConfigurationEntry Configuration])
   (:import [org.mockito Mockito])
   (:use [clojure test])
+  (:use [backtype.storm bootstrap])
 )
 
+(bootstrap)
+
 (deftest test-throws-on-missing-section
   (is (thrown? IOException
     (AuthUtils/get (Mockito/mock Configuration) "bogus-section" "")))
@@ -61,5 +64,16 @@
       (is (not (nil? (AuthUtils/get conf section k))))
       (is (= (AuthUtils/get conf section k) expected))
     )
+  ))
+
+(deftest test-empty-auto-creds
+  (let [result (AuthUtils/GetAutoCredentials {})]
+    (is (.isEmpty result))
+  )
+)
+
+(deftest test-empty-creds-renewers
+  (let [result (AuthUtils/GetCredentialRenewers {})]
+    (is (.isEmpty result))
   )
 )

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/backtype/storm/security/auth/DefaultHttpCredentialsPlugin_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/security/auth/DefaultHttpCredentialsPlugin_test.clj b/storm-core/test/clj/backtype/storm/security/auth/DefaultHttpCredentialsPlugin_test.clj
new file mode 100644
index 0000000..ab54d82
--- /dev/null
+++ b/storm-core/test/clj/backtype/storm/security/auth/DefaultHttpCredentialsPlugin_test.clj
@@ -0,0 +1,40 @@
+(ns backtype.storm.security.auth.DefaultHttpCredentialsPlugin-test
+  (:use [clojure test])
+  (:import [javax.security.auth Subject])
+  (:import [javax.servlet.http HttpServletRequest])
+  (:import [org.mockito Mockito])
+  (:import [backtype.storm.security.auth DefaultHttpCredentialsPlugin
+            ReqContext SingleUserPrincipal])
+  )
+
+(deftest test-getUserName
+  (let [handler (doto (DefaultHttpCredentialsPlugin.) (.prepare {}))]
+    (testing "returns null when request is null"
+      (is (nil? (.getUserName handler nil))))
+
+    (testing "returns null when user principal is null"
+      (let [req (Mockito/mock HttpServletRequest)]
+        (is (nil? (.getUserName handler req)))))
+
+    (testing "returns null when user is blank"
+      (let [princ (SingleUserPrincipal. "")
+            req (Mockito/mock HttpServletRequest)]
+        (. (Mockito/when (. req getUserPrincipal))
+           thenReturn princ)
+        (is (nil? (.getUserName handler req)))))
+
+    (testing "returns correct user from requests principal"
+      (let [exp-name "Alice"
+            princ (SingleUserPrincipal. exp-name)
+            req (Mockito/mock HttpServletRequest)]
+        (. (Mockito/when (. req getUserPrincipal))
+           thenReturn princ)
+        (is (.equals exp-name (.getUserName handler req)))))))
+
+(deftest test-populate-req-context-noop-on-null-user
+  (let [req (Mockito/mock HttpServletRequest)
+        handler (doto (DefaultHttpCredentialsPlugin.) (.prepare {}))
+        expected-subj (Subject.)
+        context (ReqContext. expected-subj)]
+    (is (.equals expected-subj
+                 (-> handler (.populateContext context req) (.subject))))))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/backtype/storm/security/auth/ThriftClient_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/security/auth/ThriftClient_test.clj b/storm-core/test/clj/backtype/storm/security/auth/ThriftClient_test.clj
index 7dcd86d..2eee963 100644
--- a/storm-core/test/clj/backtype/storm/security/auth/ThriftClient_test.clj
+++ b/storm-core/test/clj/backtype/storm/security/auth/ThriftClient_test.clj
@@ -14,28 +14,32 @@
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
 (ns backtype.storm.security.auth.ThriftClient-test
-  (:use [backtype.storm config])
+  (:use [backtype.storm config util])
   (:use [clojure test])
-  (:import [backtype.storm.security.auth ThriftClient])
+  (:import [backtype.storm.security.auth ThriftClient ThriftConnectionType])
   (:import [org.apache.thrift.transport TTransportException])
 )
 
 (deftest test-ctor-throws-if-port-invalid
-  (let [conf (read-default-config)
+  (let [conf (merge
+              (read-default-config)
+              {STORM-NIMBUS-RETRY-TIMES 0})
         timeout (Integer. 30)]
-    (is (thrown? java.lang.IllegalArgumentException
-      (ThriftClient. conf "bogushost" -1 timeout)))
-    (is (thrown? java.lang.IllegalArgumentException
-        (ThriftClient. conf "bogushost" 0 timeout)))
+    (is (thrown-cause? java.lang.IllegalArgumentException
+      (ThriftClient. conf ThriftConnectionType/DRPC "bogushost" (int -1) timeout)))
+    (is (thrown-cause? java.lang.IllegalArgumentException
+        (ThriftClient. conf ThriftConnectionType/DRPC "bogushost" (int 0) timeout)))
   )
 )
 
 (deftest test-ctor-throws-if-host-not-set
-  (let [conf (read-default-config)
+  (let [conf (merge
+              (read-default-config)
+              {STORM-NIMBUS-RETRY-TIMES 0})
         timeout (Integer. 60)]
-    (is (thrown? TTransportException
-         (ThriftClient. conf "" 4242 timeout)))
-    (is (thrown? IllegalArgumentException
-        (ThriftClient. conf nil 4242 timeout)))
+    (is (thrown-cause? TTransportException
+         (ThriftClient. conf ThriftConnectionType/DRPC "" (int 4242) timeout)))
+    (is (thrown-cause? IllegalArgumentException
+        (ThriftClient. conf ThriftConnectionType/DRPC nil (int 4242) timeout)))
   )
 )

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/backtype/storm/security/auth/ThriftServer_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/security/auth/ThriftServer_test.clj b/storm-core/test/clj/backtype/storm/security/auth/ThriftServer_test.clj
index 6213d4f..c8ed70e 100644
--- a/storm-core/test/clj/backtype/storm/security/auth/ThriftServer_test.clj
+++ b/storm-core/test/clj/backtype/storm/security/auth/ThriftServer_test.clj
@@ -16,14 +16,16 @@
 (ns backtype.storm.security.auth.ThriftServer-test
   (:use [backtype.storm config])
   (:use [clojure test])
-  (:import [backtype.storm.security.auth ThriftServer])
+  (:import [backtype.storm.security.auth ThriftServer ThriftConnectionType])
   (:import [org.apache.thrift.transport TTransportException])
 )
 
 (deftest test-stop-checks-for-null
-  (let [server (ThriftServer. (read-default-config) nil 12345)]
+  (let [server (ThriftServer. (read-default-config) nil 
+                              ThriftConnectionType/DRPC)]
     (.stop server)))
 
 (deftest test-isServing-checks-for-null
-  (let [server (ThriftServer. (read-default-config) nil 12345)]
+  (let [server (ThriftServer. (read-default-config) nil 
+                              ThriftConnectionType/DRPC)]
     (is (not (.isServing server)))))


[10/14] STORM-216: Added Authentication and Authorization.

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/generated/DistributedRPCInvocations.java
----------------------------------------------------------------------
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 class DistributedRPCInvocations {
 
   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 @@ public class DistributedRPCInvocations {
       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 @@ public class DistributedRPCInvocations {
       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 @@ public class DistributedRPCInvocations {
       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 @@ public class DistributedRPCInvocations {
       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 @@ public class DistributedRPCInvocations {
         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 @@ public class DistributedRPCInvocations {
         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 @@ public class DistributedRPCInvocations {
         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 @@ public class DistributedRPCInvocations {
 
       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 @@ public class DistributedRPCInvocations {
 
       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 @@ public class DistributedRPCInvocations {
 
       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 class DistributedRPCInvocations {
   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 class DistributedRPCInvocations {
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
+          case 1: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -784,9 +809,14 @@ public class DistributedRPCInvocations {
         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 class DistributedRPCInvocations {
     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 @@ public class DistributedRPCInvocations {
 
     @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 @@ public class DistributedRPCInvocations {
       }
 
       switch (field) {
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -843,6 +920,15 @@ public class DistributedRPCInvocations {
       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 class DistributedRPCInvocations {
     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 @@ public class DistributedRPCInvocations {
       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 @@ public class DistributedRPCInvocations {
           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 class DistributedRPCInvocations {
     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 @@ public class DistributedRPCInvocations {
       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 @@ public class DistributedRPCInvocations {
     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 @@ public class DistributedRPCInvocations {
         switch(fieldId) {
           case 0: // SUCCESS
             return SUCCESS;
+          case 1: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -1298,6 +1424,8 @@ public class DistributedRPCInvocations {
       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 class DistributedRPCInvocations {
     }
 
     public fetchRequest_result(
-      DRPCRequest success)
+      DRPCRequest success,
+      AuthorizationException aze)
     {
       this();
       this.success = success;
+      this.aze = aze;
     }
 
     /**
@@ -1319,6 +1449,9 @@ public class DistributedRPCInvocations {
       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 @@ public class DistributedRPCInvocations {
     @Override
     public void clear() {
       this.success = null;
+      this.aze = null;
     }
 
     public DRPCRequest get_success() {
@@ -1353,6 +1487,29 @@ public class DistributedRPCInvocations {
       }
     }
 
+    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 @@ public class DistributedRPCInvocations {
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -1371,6 +1536,9 @@ public class DistributedRPCInvocations {
       case SUCCESS:
         return get_success();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -1384,6 +1552,8 @@ public class DistributedRPCInvocations {
       switch (field) {
       case SUCCESS:
         return is_set_success();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -1410,6 +1580,15 @@ public class DistributedRPCInvocations {
           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 @@ public class DistributedRPCInvocations {
       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 @@ public class DistributedRPCInvocations {
           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 @@ public class DistributedRPCInvocations {
               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 @@ public class DistributedRPCInvocations {
         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 @@ public class DistributedRPCInvocations {
         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 class DistributedRPCInvocations {
   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 class DistributedRPCInvocations {
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
+          case 1: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -1888,9 +2106,14 @@ public class DistributedRPCInvocations {
         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 class DistributedRPCInvocations {
     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 @@ public class DistributedRPCInvocations {
 
     @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 @@ public class DistributedRPCInvocations {
       }
 
       switch (field) {
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -1947,6 +2217,15 @@ public class DistributedRPCInvocations {
       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 class DistributedRPCInvocations {
     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 @@ public class DistributedRPCInvocations {
       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 @@ public class DistributedRPCInvocations {
           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 class DistributedRPCInvocations {
     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 @@ public class DistributedRPCInvocations {
       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();
     }


[11/14] STORM-216: Added Authentication and Authorization.

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/ConfigValidation.java
----------------------------------------------------------------------
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 class ConfigValidation {
          */
         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.
+     * 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
+     */
+    public static NestableFieldValidator fv(final Class cls, final boolean nullAllowed) {
+        return new NestableFieldValidator() {
+            @Override
+            public void validateField(String pd, String name, Object field)
+                    throws IllegalArgumentException {
+                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
-     * @return a FieldValidator for a list of the given class
+     * @param nullAllowed whether or not a value of null is valid
+     * @return a NestableFieldValidator for a list of the given class
      */
-    static FieldValidator FieldListValidatorFactory(final Class cls) {
-        return new FieldValidator() {
+    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 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 (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 = listFv(String.class, true);
+    
+    /**
+     * Validates a map of Strings to Numbers.
      */
-    public static Object StringsValidator = FieldListValidatorFactory(String.class);
+    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 @@ public class ConfigValidation {
     };
 
     /**
+     * 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 class ConfigValidation {
      */
     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 {

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/Constants.java
----------------------------------------------------------------------
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 class Constants {
     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
+    

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/ICredentialsListener.java
----------------------------------------------------------------------
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);
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/ILocalCluster.java
----------------------------------------------------------------------
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.NotAliveException;
 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 @@ import java.util.Map;
 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;

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/StormSubmitter.java
----------------------------------------------------------------------
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 @@ package backtype.storm;
 
 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.json.simple.JSONValue;
 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 @@ public class StormSubmitter {
 
     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 @@ public class StormSubmitter {
      * @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 @@ public class StormSubmitter {
      * @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 @@ public class StormSubmitter {
         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 @@ public class StormSubmitter {
      * @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 @@ public class StormSubmitter {
      * @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 @@ public class StormSubmitter {
         }
     }
 
-    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);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/drpc/DRPCInvocationsClient.java
----------------------------------------------------------------------
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);
-        }
-    }
-    
-    private void connect() throws TException {
-        conn = new TFramedTransport(new TSocket(host, port));
-        client = new DistributedRPCInvocations.Client(new TBinaryProtocol(conn));
-        conn.open();
+    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));
     }
-    
+        
     public String getHost() {
         return host;
     }
@@ -55,37 +52,57 @@ public class DRPCInvocationsClient implements DistributedRPCInvocations.Iface {
         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();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/drpc/DRPCSpout.java
----------------------------------------------------------------------
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.Config;
 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.ServiceRegistry;
 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 @@ public class DRPCSpout extends BaseRichSpout {
         _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 @@ public class DRPCSpout extends BaseRichSpout {
             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 class DRPCSpout extends BaseRichSpout {
     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 @@ public class DRPCSpout extends BaseRichSpout {
                         _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 @@ public class DRPCSpout extends BaseRichSpout {
                     }
                 } catch (TException e) {
                     throw new RuntimeException(e);
+                } catch (AuthorizationException aze) {
+                    throw new RuntimeException(aze);
                 }
             }
         }
@@ -159,6 +241,8 @@ public class DRPCSpout extends BaseRichSpout {
             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);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/drpc/ReturnResults.java
----------------------------------------------------------------------
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 @@ package backtype.storm.drpc;
 
 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 java.util.Map;
 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 @@ public class ReturnResults extends BaseRichBolt {
                 }};
             
                 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);
+                    }
+                }
             }
         }
     }    

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/generated/AuthorizationException.java
----------------------------------------------------------------------
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);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/generated/Credentials.java
----------------------------------------------------------------------
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);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/generated/DistributedRPC.java
----------------------------------------------------------------------
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 class DistributedRPC {
 
   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 @@ public class DistributedRPC {
       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 @@ public class DistributedRPC {
       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 @@ public class DistributedRPC {
       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 @@ public class DistributedRPC {
         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 @@ public class DistributedRPC {
           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 @@ public class DistributedRPC {
 
     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 @@ public class DistributedRPC {
             return SUCCESS;
           case 1: // E
             return E;
+          case 2: // AZE
+            return AZE;
           default:
             return null;
         }
@@ -664,6 +674,8 @@ public class DistributedRPC {
           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 class DistributedRPC {
 
     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 @@ public class DistributedRPC {
       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 class DistributedRPC {
     public void clear() {
       this.success = null;
       this.e = null;
+      this.aze = null;
     }
 
     public String get_success() {
@@ -748,6 +766,29 @@ public class DistributedRPC {
       }
     }
 
+    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 @@ public class DistributedRPC {
         }
         break;
 
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
       }
     }
 
@@ -777,6 +826,9 @@ public class DistributedRPC {
       case E:
         return get_e();
 
+      case AZE:
+        return get_aze();
+
       }
       throw new IllegalStateException();
     }
@@ -792,6 +844,8 @@ public class DistributedRPC {
         return is_set_success();
       case E:
         return is_set_e();
+      case AZE:
+        return is_set_aze();
       }
       throw new IllegalStateException();
     }
@@ -827,6 +881,15 @@ public class DistributedRPC {
           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 @@ public class DistributedRPC {
       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 @@ public class DistributedRPC {
           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 @@ public class DistributedRPC {
               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 @@ public class DistributedRPC {
         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 @@ public class DistributedRPC {
         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();
     }


[07/14] STORM-216: Added Authentication and Authorization.

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java b/storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java
index b0e48e5..b5118c7 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java
@@ -20,6 +20,7 @@ package backtype.storm.security.auth;
 import backtype.storm.Config;
 import javax.security.auth.login.Configuration;
 import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.Subject;
 import java.security.NoSuchAlgorithmException;
 import java.security.URIParameter;
 import org.slf4j.Logger;
@@ -28,7 +29,11 @@ import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
+import java.util.Collection;
+import java.util.Set;
+import java.util.HashSet;
 import java.util.Map;
+import java.util.concurrent.ExecutorService;
 
 public class AuthUtils {
     private static final Logger LOG = LoggerFactory.getLogger(AuthUtils.class);
@@ -47,15 +52,16 @@ public class AuthUtils {
         //find login file configuration from Storm configuration  
         String loginConfigurationFile = (String)storm_conf.get("java.security.auth.login.config");
         if ((loginConfigurationFile != null) && (loginConfigurationFile.length()>0)) { 
+            File config_file = new File(loginConfigurationFile);
+            if (! config_file.canRead()) {
+                throw new RuntimeException("File " + loginConfigurationFile +
+                        " cannot be read.");
+            }
             try {
-                URI config_uri = new File(loginConfigurationFile).toURI();
+                URI config_uri = config_file.toURI();
                 login_conf = Configuration.getInstance("JavaLoginConfig", new URIParameter(config_uri));
-            } catch (NoSuchAlgorithmException ex1) {
-                if (ex1.getCause() instanceof FileNotFoundException)
-                    throw new RuntimeException("configuration file "+loginConfigurationFile+" could not be found");
-                else throw new RuntimeException(ex1);
-            } catch (Exception ex2) {
-                throw new RuntimeException(ex2);
+            } catch (Exception ex) {
+                throw new RuntimeException(ex);
             }
         }
         
@@ -63,23 +69,162 @@ public class AuthUtils {
     }
 
     /**
+     * Construct a principal to local plugin
+     * @param conf storm configuration
+     * @return the plugin
+     */
+    public static IPrincipalToLocal GetPrincipalToLocalPlugin(Map storm_conf) {
+        IPrincipalToLocal ptol = null;
+        try {
+          String ptol_klassName = (String) storm_conf.get(Config.STORM_PRINCIPAL_TO_LOCAL_PLUGIN);
+          Class klass = Class.forName(ptol_klassName);
+          ptol = (IPrincipalToLocal)klass.newInstance();
+          ptol.prepare(storm_conf);
+        } catch (Exception e) {
+          throw new RuntimeException(e);
+        }
+        return ptol;
+    }
+
+    /**
+     * Get all of the configured Credential Renwer Plugins.
+     * @param storm_conf the storm configuration to use.
+     * @return the configured credential renewers.
+     */ 
+    public static Collection<ICredentialsRenewer> GetCredentialRenewers(Map conf) {
+        try {
+            Set<ICredentialsRenewer> ret = new HashSet<ICredentialsRenewer>();
+            Collection<String> clazzes = (Collection<String>)conf.get(Config.NIMBUS_CREDENTIAL_RENEWERS);
+            if (clazzes != null) {
+                for (String clazz : clazzes) {
+                    ICredentialsRenewer inst = (ICredentialsRenewer)Class.forName(clazz).newInstance();
+                    inst.prepare(conf);
+                    ret.add(inst);
+                }
+            }
+            return ret;
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * Get all of the configured AutoCredential Plugins.
+     * @param storm_conf the storm configuration to use.
+     * @return the configured auto credentials.
+     */ 
+    public static Collection<IAutoCredentials> GetAutoCredentials(Map storm_conf) {
+        try {
+            Set<IAutoCredentials> autos = new HashSet<IAutoCredentials>();
+            Collection<String> clazzes = (Collection<String>)storm_conf.get(Config.TOPOLOGY_AUTO_CREDENTIALS);
+            if (clazzes != null) {
+                for (String clazz : clazzes) {
+                    IAutoCredentials a = (IAutoCredentials)Class.forName(clazz).newInstance();
+                    a.prepare(storm_conf);
+                    autos.add(a);
+                }
+            }
+            LOG.info("Got AutoCreds "+autos);
+            return autos;
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * Populate a subject from credentials using the IAutoCredentials.
+     * @param subject the subject to populate or null if a new Subject should be created.
+     * @param autos the IAutoCredentials to call to populate the subject.
+     * @param credentials the credentials to pull from
+     * @return the populated subject.
+     */ 
+    public static Subject populateSubject(Subject subject, Collection<IAutoCredentials> autos, Map<String,String> credentials) {
+        try {
+            if (subject == null) {
+                subject = new Subject();
+            }
+            for (IAutoCredentials autoCred : autos) {
+                autoCred.populateSubject(subject, credentials);
+            }
+            return subject;
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * Update a subject from credentials using the IAutoCredentials.
+     * @param subject the subject to update 
+     * @param autos the IAutoCredentials to call to update the subject.
+     * @param credentials the credentials to pull from
+     */ 
+    public static void updateSubject(Subject subject, Collection<IAutoCredentials> autos, Map<String,String> credentials) {
+        if (subject == null) {
+            throw new RuntimeException("The subject cannot be null when updating a subject with credentials");
+        }
+
+        try {
+            for (IAutoCredentials autoCred : autos) {
+                autoCred.updateSubject(subject, credentials);
+            }
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
      * Construct a transport plugin per storm configuration
      * @param conf storm configuration
      * @return
      */
-    public static ITransportPlugin GetTransportPlugin(Map storm_conf, Configuration login_conf) {
+    public static ITransportPlugin GetTransportPlugin(ThriftConnectionType type, Map storm_conf, Configuration login_conf) {
         ITransportPlugin  transportPlugin = null;
         try {
-            String transport_plugin_klassName = (String) storm_conf.get(Config.STORM_THRIFT_TRANSPORT_PLUGIN);
+            String transport_plugin_klassName = type.getTransportPlugin(storm_conf);
             Class klass = Class.forName(transport_plugin_klassName);
             transportPlugin = (ITransportPlugin)klass.newInstance();
-            transportPlugin.prepare(storm_conf, login_conf);
+            transportPlugin.prepare(type, storm_conf, login_conf);
         } catch(Exception e) {
             throw new RuntimeException(e);
         } 
         return transportPlugin;
     }
 
+    private static IHttpCredentialsPlugin GetHttpCredentialsPlugin(Map conf,
+            String klassName) {
+        IHttpCredentialsPlugin plugin = null;
+        try {
+            Class klass = Class.forName(klassName);
+            plugin = (IHttpCredentialsPlugin)klass.newInstance();
+            plugin.prepare(conf);
+        } catch(Exception e) {
+            throw new RuntimeException(e);
+        }
+        return plugin;
+    }
+
+    /**
+     * Construct an HttpServletRequest credential plugin specified by the UI
+     * storm configuration
+     * @param conf storm configuration
+     * @return the plugin
+     */
+    public static IHttpCredentialsPlugin GetUiHttpCredentialsPlugin(Map conf) {
+        String klassName = (String)conf.get(Config.UI_HTTP_CREDS_PLUGIN);
+        return AuthUtils.GetHttpCredentialsPlugin(conf, klassName);
+    }
+
+    /**
+     * Construct an HttpServletRequest credential plugin specified by the DRPC
+     * storm configuration
+     * @param conf storm configuration
+     * @return the plugin
+     */
+    public static IHttpCredentialsPlugin GetDrpcHttpCredentialsPlugin(Map conf) {
+        String klassName = (String)conf.get(Config.DRPC_HTTP_CREDS_PLUGIN);
+        return AuthUtils.GetHttpCredentialsPlugin(conf, klassName);
+    }
+
     public static String get(Configuration configuration, String section, String key) throws IOException {
         AppConfigurationEntry configurationEntries[] = configuration.getAppConfigurationEntry(section);
         if (configurationEntries == null) {

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/DefaultHttpCredentialsPlugin.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/DefaultHttpCredentialsPlugin.java b/storm-core/src/jvm/backtype/storm/security/auth/DefaultHttpCredentialsPlugin.java
new file mode 100644
index 0000000..7d8e0fb
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/security/auth/DefaultHttpCredentialsPlugin.java
@@ -0,0 +1,83 @@
+/**
+ * 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.security.auth;
+
+import java.security.Principal;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import javax.security.auth.Subject;
+import javax.servlet.http.HttpServletRequest;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import backtype.storm.security.auth.ReqContext;
+
+public class DefaultHttpCredentialsPlugin implements IHttpCredentialsPlugin {
+    private static final Logger LOG =
+            LoggerFactory.getLogger(DefaultHttpCredentialsPlugin.class);
+
+    /**
+     * No-op
+     * @param storm_conf Storm configuration
+     */
+    @Override
+    public void prepare(Map storm_conf) {
+        // Do nothing.
+    }
+
+    /**
+     * Gets the user name from the request principal.
+     * @param req the servlet request
+     * @return the authenticated user, or null if none is authenticated
+     */
+    @Override
+    public String getUserName(HttpServletRequest req) {
+        Principal princ = null;
+        if (req != null && (princ = req.getUserPrincipal()) != null) {
+            String userName = princ.getName();
+            if (userName != null && !userName.isEmpty()) {
+                LOG.debug("HTTP request had user ("+userName+")");
+                return userName;
+            }
+        }
+        return null;
+    }
+
+    /**
+     * Populates a given context with a new Subject derived from the
+     * credentials in a servlet request.
+     * @param context the context to be populated
+     * @param req the servlet request
+     * @return the context
+     */
+    @Override
+    public ReqContext populateContext(ReqContext context,
+            HttpServletRequest req) {
+        String userName = getUserName(req);
+        if (userName != null) {
+            Set<SingleUserPrincipal> principals = new HashSet<SingleUserPrincipal>(1);
+            principals.add(new SingleUserPrincipal(userName));
+            Subject s = new Subject(true, principals, new HashSet(), new HashSet());
+            context.setSubject(s);
+        }
+        return context;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/DefaultPrincipalToLocal.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/DefaultPrincipalToLocal.java b/storm-core/src/jvm/backtype/storm/security/auth/DefaultPrincipalToLocal.java
new file mode 100644
index 0000000..729d744
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/security/auth/DefaultPrincipalToLocal.java
@@ -0,0 +1,43 @@
+/**
+ * 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.security.auth;
+
+import java.util.Map;
+import java.security.Principal;
+
+/**
+ * Storm can be configured to launch worker processed as a given user.
+ * Some transports need to map the Principal to a local user name.
+ */
+public class DefaultPrincipalToLocal implements IPrincipalToLocal {
+    /**
+     * Invoked once immediately after construction
+     * @param conf Storm configuration 
+     */
+    public void prepare(Map storm_conf) {}
+    
+    /**
+     * Convert a Principal to a local user name.
+     * @param principal the principal to convert
+     * @return The local user name.
+     */
+    public String toLocal(Principal principal) {
+      return principal == null ? null : principal.getName();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/IAutoCredentials.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/IAutoCredentials.java b/storm-core/src/jvm/backtype/storm/security/auth/IAutoCredentials.java
new file mode 100644
index 0000000..b3886da
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/security/auth/IAutoCredentials.java
@@ -0,0 +1,55 @@
+/**
+ * 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.security.auth;
+
+import java.util.Map;
+
+import javax.security.auth.Subject;
+
+/**
+ * Provides a way to automatically push credentials to a topology and to
+ * retreave them in the worker.
+ */
+public interface IAutoCredentials {
+
+    public void prepare(Map conf);
+
+    /**
+     * Called to populate the credentials on the client side.
+     * @param credentials the credentials to be populated.
+     */
+    public void populateCredentials(Map<String, String> credentials);
+
+    /**
+     * Called to initially populate the subject on the worker side with credentials passed in.
+     * @param subject the subject to optionally put credentials in.
+     * @param credentials the credentials to be used.
+     */ 
+    public void populateSubject(Subject subject, Map<String, String> credentials);
+
+
+    /**
+     * Called to update the subject on the worker side when new credentials are recieved.
+     * This means that populateSubject has already been called on this subject.  
+     * @param subject the subject to optionally put credentials in.
+     * @param credentials the credentials to be used.
+     */ 
+    public void updateSubject(Subject subject, Map<String, String> credentials);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/ICredentialsRenewer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/ICredentialsRenewer.java b/storm-core/src/jvm/backtype/storm/security/auth/ICredentialsRenewer.java
new file mode 100644
index 0000000..a84a9e8
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/security/auth/ICredentialsRenewer.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package backtype.storm.security.auth;
+
+import java.util.Collection;
+import java.util.Map;
+
+/**
+ * Provides a way to renew credentials on behelf of a user.
+ */
+public interface ICredentialsRenewer {
+
+   /**
+    * Called when initializing the service.
+    * @param conf the storm cluster configuration.
+    */ 
+   public void prepare(Map conf);
+
+    /**
+     * Renew any credentials that need to be renewed. (Update the credentials if needed)
+     * @param credentials the credentials that may have something to renew.
+     */ 
+    public void renew(Map<String, String> credentials);
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/IHttpCredentialsPlugin.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/IHttpCredentialsPlugin.java b/storm-core/src/jvm/backtype/storm/security/auth/IHttpCredentialsPlugin.java
new file mode 100644
index 0000000..a012ce4
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/security/auth/IHttpCredentialsPlugin.java
@@ -0,0 +1,50 @@
+/**
+ * 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.security.auth;
+
+import java.util.Map;
+import javax.servlet.http.HttpServletRequest;
+
+import backtype.storm.security.auth.ReqContext;
+
+/**
+ * Interface for handling credentials in an HttpServletRequest
+ */
+public interface IHttpCredentialsPlugin {
+    /**
+     * Invoked once immediately after construction
+     * @param storm_conf Storm configuration
+     */
+    void prepare(Map storm_conf);
+
+    /**
+     * Gets the user name from the request.
+     * @param req the servlet request
+     * @return the authenticated user, or null if none is authenticated.
+     */
+    String getUserName(HttpServletRequest req);
+
+    /**
+     * Populates a given context with credentials information from an HTTP
+     * request.
+     * @param req the servlet request
+     * @return the context
+     */
+    ReqContext populateContext(ReqContext context, HttpServletRequest req);
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/IPrincipalToLocal.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/IPrincipalToLocal.java b/storm-core/src/jvm/backtype/storm/security/auth/IPrincipalToLocal.java
new file mode 100644
index 0000000..fca3d37
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/security/auth/IPrincipalToLocal.java
@@ -0,0 +1,41 @@
+/**
+ * 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.security.auth;
+
+import java.util.Map;
+import java.security.Principal;
+
+/**
+ * Storm can be configured to launch worker processed as a given user.
+ * Some transports need to map the Principal to a local user name.
+ */
+public interface IPrincipalToLocal {
+    /**
+     * Invoked once immediately after construction
+     * @param conf Storm configuration 
+     */
+    void prepare(Map storm_conf);
+    
+    /**
+     * Convert a Principal to a local user name.
+     * @param principal the principal to convert
+     * @return The local user name.
+     */
+    public String toLocal(Principal principal);
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/ITransportPlugin.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/ITransportPlugin.java b/storm-core/src/jvm/backtype/storm/security/auth/ITransportPlugin.java
index a44d115..7575d71 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/ITransportPlugin.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/ITransportPlugin.java
@@ -18,7 +18,9 @@
 package backtype.storm.security.auth;
 
 import java.io.IOException;
+import java.security.Principal;
 import java.util.Map;
+import java.util.concurrent.ExecutorService;
 
 import javax.security.auth.login.Configuration;
 
@@ -27,24 +29,26 @@ import org.apache.thrift.server.TServer;
 import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportException;
 
+import backtype.storm.security.auth.ThriftConnectionType;
+
 /**
  * Interface for Thrift Transport plugin
  */
 public interface ITransportPlugin {
     /**
      * Invoked once immediately after construction
+     * @param type the type of connection this will process.
      * @param storm_conf Storm configuration 
      * @param login_conf login configuration
      */
-    void prepare(Map storm_conf, Configuration login_conf);
+    void prepare(ThriftConnectionType type, Map storm_conf, Configuration login_conf);
     
     /**
-     * Create a server associated with a given port and service handler
-     * @param port listening port
+     * Create a server associated with a given port, service handler, and purpose
      * @param processor service handler
-     * @return server to be binded
+     * @return server
      */
-    public TServer getServer(int port, TProcessor processor) throws IOException, TTransportException;
+    public TServer getServer(TProcessor processor) throws IOException, TTransportException;
 
     /**
      * Connect to the specified server via framed transport 

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/KerberosPrincipalToLocal.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/KerberosPrincipalToLocal.java b/storm-core/src/jvm/backtype/storm/security/auth/KerberosPrincipalToLocal.java
new file mode 100644
index 0000000..35c7788
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/security/auth/KerberosPrincipalToLocal.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package backtype.storm.security.auth;
+
+import java.util.Map;
+import java.security.Principal;
+
+/**
+ * Map a kerberos principal to a local user
+ */
+public class KerberosPrincipalToLocal implements IPrincipalToLocal {
+
+    /**
+     * Invoked once immediately after construction
+     * @param conf Storm configuration 
+     */
+    public void prepare(Map storm_conf) {}
+    
+    /**
+     * Convert a Principal to a local user name.
+     * @param principal the principal to convert
+     * @return The local user name.
+     */
+    public String toLocal(Principal principal) {
+      //This technically does not conform with rfc1964, but should work so
+      // long as you don't have any really odd names in your KDC.
+      return principal == null ? null : principal.getName().split("[/@]")[0];
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/ReqContext.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/ReqContext.java b/storm-core/src/jvm/backtype/storm/security/auth/ReqContext.java
index 68d8493..4033f18 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/ReqContext.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/ReqContext.java
@@ -59,11 +59,19 @@ public class ReqContext {
 
     //private constructor
     @VisibleForTesting
-    ReqContext(AccessControlContext acl_ctxt) {
+    public ReqContext(AccessControlContext acl_ctxt) {
         _subject = Subject.getSubject(acl_ctxt);
         _reqID = uniqueId.incrementAndGet();
     }
 
+    //private constructor
+    @VisibleForTesting
+    public ReqContext(Subject sub) {
+        _subject = sub;
+        _reqID = uniqueId.incrementAndGet();
+    }
+
+
     /**
      * client address
      */

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/SaslTransportPlugin.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/SaslTransportPlugin.java b/storm-core/src/jvm/backtype/storm/security/auth/SaslTransportPlugin.java
index 44ab754..348fae3 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/SaslTransportPlugin.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/SaslTransportPlugin.java
@@ -21,6 +21,9 @@ import java.io.IOException;
 import java.net.Socket;
 import java.security.Principal;
 import java.util.Map;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.TimeUnit;
 
 import javax.security.auth.Subject;
 import javax.security.auth.login.Configuration;
@@ -40,38 +43,48 @@ import org.apache.thrift.transport.TTransportFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import backtype.storm.security.auth.ThriftConnectionType;
+
 /**
  * Base class for SASL authentication plugin.
  */
 public abstract class SaslTransportPlugin implements ITransportPlugin {
+    protected ThriftConnectionType type;
+    protected Map storm_conf;
     protected Configuration login_conf;
     private static final Logger LOG = LoggerFactory.getLogger(SaslTransportPlugin.class);
 
-    /**
-     * Invoked once immediately after construction
-     * @param conf Storm configuration 
-     * @param login_conf login configuration
-     */
-    public void prepare(Map storm_conf, Configuration login_conf) {        
+    @Override
+    public void prepare(ThriftConnectionType type, Map storm_conf, Configuration login_conf) {
+        this.type = type;
+        this.storm_conf = storm_conf;
         this.login_conf = login_conf;
     }
 
-    public TServer getServer(int port, TProcessor processor) throws IOException, TTransportException {
+    @Override
+    public TServer getServer(TProcessor processor) throws IOException, TTransportException {
+        int port = type.getPort(storm_conf);
         TTransportFactory serverTransportFactory = getServerTransportFactory();
-
-        //define THsHaServer args 
-        //original: THsHaServer + TNonblockingServerSocket
-        //option: TThreadPoolServer + TServerSocket
         TServerSocket serverTransport = new TServerSocket(port);
+        int numWorkerThreads = type.getNumThreads(storm_conf);
+        int maxBufferSize = type.getMaxBufferSize(storm_conf);
+        Integer queueSize = type.getQueueSize(storm_conf);
+
         TThreadPoolServer.Args server_args = new TThreadPoolServer.Args(serverTransport).
                 processor(new TUGIWrapProcessor(processor)).
-                minWorkerThreads(64).
-                maxWorkerThreads(64).
-                protocolFactory(new TBinaryProtocol.Factory());            
-        if (serverTransportFactory != null) 
+                minWorkerThreads(numWorkerThreads).
+                maxWorkerThreads(numWorkerThreads).
+                protocolFactory(new TBinaryProtocol.Factory(false, true, maxBufferSize));
+
+        if (serverTransportFactory != null) {
             server_args.transportFactory(serverTransportFactory);
+        }
+
+        //if (queueSize != null) {
+        //    server_args.executorService(new ThreadPoolExecutor(numWorkerThreads, numWorkerThreads, 
+        //                           60, TimeUnit.SECONDS, new ArrayBlockingQueue(queueSize)));
+        //}
 
-        //construct THsHaServer
         return new TThreadPoolServer(server_args);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/SimpleTransportPlugin.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/SimpleTransportPlugin.java b/storm-core/src/jvm/backtype/storm/security/auth/SimpleTransportPlugin.java
index 5a81746..f0905e2 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/SimpleTransportPlugin.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/SimpleTransportPlugin.java
@@ -18,12 +18,18 @@
 package backtype.storm.security.auth;
 
 import java.io.IOException;
+import java.security.Principal;
 import java.net.InetAddress;
 import java.net.Socket;
 import java.net.UnknownHostException;
 import java.util.Map;
+import java.util.HashSet;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.TimeUnit;
 
 import javax.security.auth.login.Configuration;
+import javax.security.auth.Subject;
 import org.apache.thrift.TException;
 import org.apache.thrift.TProcessor;
 import org.apache.thrift.protocol.TBinaryProtocol;
@@ -39,33 +45,43 @@ import org.apache.thrift.transport.TTransportException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import backtype.storm.security.auth.ThriftConnectionType;
+
 /**
  * Simple transport for Thrift plugin.
  * 
  * This plugin is designed to be backward compatible with existing Storm code.
  */
 public class SimpleTransportPlugin implements ITransportPlugin {
+    protected ThriftConnectionType type;
+    protected Map storm_conf;
     protected Configuration login_conf;
     private static final Logger LOG = LoggerFactory.getLogger(SimpleTransportPlugin.class);
 
-    /**
-     * Invoked once immediately after construction
-     * @param conf Storm configuration 
-     * @param login_conf login configuration
-     */
-    public void prepare(Map storm_conf, Configuration login_conf) {        
+    @Override
+    public void prepare(ThriftConnectionType type, Map storm_conf, Configuration login_conf) {
+        this.type = type;
+        this.storm_conf = storm_conf;
         this.login_conf = login_conf;
     }
 
-    /**
-     * We will let Thrift to apply default transport factory
-     */
-    public TServer getServer(int port, TProcessor processor) throws IOException, TTransportException {
+    @Override
+    public TServer getServer(TProcessor processor) throws IOException, TTransportException {
+        int port = type.getPort(storm_conf);
         TNonblockingServerSocket serverTransport = new TNonblockingServerSocket(port);
+        int numWorkerThreads = type.getNumThreads(storm_conf);
+        int maxBufferSize = type.getMaxBufferSize(storm_conf);
+        Integer queueSize = type.getQueueSize(storm_conf);
+
         THsHaServer.Args server_args = new THsHaServer.Args(serverTransport).
                 processor(new SimpleWrapProcessor(processor)).
-                workerThreads(64).
-                protocolFactory(new TBinaryProtocol.Factory());            
+                workerThreads(numWorkerThreads).
+                protocolFactory(new TBinaryProtocol.Factory(false, true, maxBufferSize));
+
+        if (queueSize != null) {
+            server_args.executorService(new ThreadPoolExecutor(numWorkerThreads, numWorkerThreads, 
+                                   60, TimeUnit.SECONDS, new ArrayBlockingQueue(queueSize)));
+        }
 
         //construct THsHaServer
         return new THsHaServer(server_args);
@@ -86,6 +102,13 @@ public class SimpleTransportPlugin implements ITransportPlugin {
         return conn;
     }
 
+    /**
+     * @return the subject that will be used for all connections
+     */  
+    protected Subject getDefaultSubject() {
+        return null;
+    }
+
     /**                                                                                                                                                                             
      * Processor that populate simple transport info into ReqContext, and then invoke a service handler                                                                              
      */
@@ -115,7 +138,19 @@ public class SimpleTransportPlugin implements ITransportPlugin {
             } 
 
             //anonymous user
-            req_context.setSubject(null);
+            Subject s = getDefaultSubject();
+            if (s == null) {
+              final String user = (String)storm_conf.get("debug.simple.transport.user");
+              if (user != null) {
+                HashSet<Principal> principals = new HashSet<Principal>();
+                principals.add(new Principal() {
+                  public String getName() { return user; }
+                  public String toString() { return user; }
+                });
+                s = new Subject(true, principals, new HashSet<Object>(), new HashSet<Object>());
+              }
+            }
+            req_context.setSubject(s);
 
             //invoke service handler
             return wrapped.process(inProt, outProt);

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/SingleUserPrincipal.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/SingleUserPrincipal.java b/storm-core/src/jvm/backtype/storm/security/auth/SingleUserPrincipal.java
new file mode 100644
index 0000000..6af17fa
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/security/auth/SingleUserPrincipal.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package backtype.storm.security.auth;
+
+import java.security.Principal;
+
+/**
+ * A Principal that represents a user.
+ */
+public class SingleUserPrincipal implements Principal {
+
+    private final String _userName;
+
+    public SingleUserPrincipal(String userName) {
+        _userName = userName;
+    }
+
+    @Override
+    public boolean equals(Object another) {
+        if (another instanceof SingleUserPrincipal) {
+            return _userName.equals(((SingleUserPrincipal)another)._userName);
+        }
+        return false;
+    }
+
+    @Override
+    public String getName() {
+        return _userName;
+    }
+
+    @Override
+    public String toString() {
+        return this.getName();
+    }
+
+    @Override
+    public int hashCode() {
+        return _userName.hashCode();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/TBackoffConnect.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/TBackoffConnect.java b/storm-core/src/jvm/backtype/storm/security/auth/TBackoffConnect.java
new file mode 100644
index 0000000..b50ae6e
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/security/auth/TBackoffConnect.java
@@ -0,0 +1,77 @@
+/**
+ * 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.security.auth;
+
+import java.io.IOException;
+import java.util.Random;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import backtype.storm.utils.Utils.BoundedExponentialBackoffRetry;
+
+public class TBackoffConnect {
+    private static final Logger LOG = LoggerFactory.getLogger(TBackoffConnect.class);
+    private int _completedRetries = 0;
+    private int _retryTimes;
+    private BoundedExponentialBackoffRetry waitGrabber;
+
+    public TBackoffConnect(int retryTimes, int retryInterval, int retryIntervalCeiling) {
+
+        _retryTimes = retryTimes;
+        waitGrabber = new BoundedExponentialBackoffRetry(retryInterval,
+                                                         retryTimes,
+                                                         retryIntervalCeiling);
+    }
+
+    public TTransport doConnectWithRetry(ITransportPlugin transportPlugin, TTransport underlyingTransport, String host) throws IOException {
+        boolean connected = false;
+        TTransport transportResult = null;
+        while(!connected) {
+            try {
+                transportResult = transportPlugin.connect(underlyingTransport, host);
+                connected = true;
+            } catch (TTransportException ex) {
+                retryNext(ex);
+            }
+        }
+        return transportResult;
+    }
+
+    private void retryNext(TTransportException ex) {
+        if(!canRetry()) {
+            throw new RuntimeException(ex);
+        }
+        try {
+            int sleeptime = waitGrabber.getSleepTimeMs(_completedRetries, 0);
+
+            LOG.debug("Failed to connect. Retrying... (" + Integer.toString( _completedRetries) + ") in " + Integer.toString(sleeptime) + "ms");
+
+            Thread.sleep(sleeptime);
+        } catch (InterruptedException e) {
+            LOG.info("Nimbus connection retry interrupted.");
+        }
+
+        _completedRetries++;
+    }
+
+    private boolean canRetry() {
+        return (_completedRetries < _retryTimes);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/ThriftClient.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/ThriftClient.java b/storm-core/src/jvm/backtype/storm/security/auth/ThriftClient.java
index d397127..ec337d9 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/ThriftClient.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/ThriftClient.java
@@ -24,56 +24,91 @@ import org.apache.thrift.protocol.TBinaryProtocol;
 import org.apache.thrift.protocol.TProtocol;
 import org.apache.thrift.transport.TSocket;
 import org.apache.thrift.transport.TTransport;
-import org.apache.thrift.transport.TTransportException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import backtype.storm.utils.Utils;
+import backtype.storm.Config;
+import backtype.storm.security.auth.TBackoffConnect;
 
 public class ThriftClient {	
     private static final Logger LOG = LoggerFactory.getLogger(ThriftClient.class);
     private TTransport _transport;
     protected TProtocol _protocol;
+    private String _host;
+    private Integer _port;
+    private Integer _timeout;
+    private Map _conf;
+    private ThriftConnectionType _type;
 
-    public ThriftClient(Map storm_conf, String host, int port) throws TTransportException {
-        this(storm_conf, host, port, null);
+    public ThriftClient(Map storm_conf, ThriftConnectionType type, String host) {
+        this(storm_conf, type, host, null, null);
     }
 
-    public ThriftClient(Map storm_conf, String host, int port, Integer timeout) throws TTransportException {
+    public ThriftClient(Map storm_conf, ThriftConnectionType type, String host, Integer port, Integer timeout) {
+        //create a socket with server
+        if (host==null) {
+            throw new IllegalArgumentException("host is not set");
+        }
+
+        if (port == null) {
+            port = type.getPort(storm_conf);
+        }
+
+        if (port<=0) {
+            throw new IllegalArgumentException("invalid port: "+port);
+        }          
+
+        _host = host;
+        _port = port;
+        _timeout = timeout;
+        _conf = storm_conf;
+        _type = type;
+        reconnect();
+    }
+
+    public synchronized TTransport transport() {
+        return _transport;
+    }
+    
+    public synchronized void reconnect() {
+        close();    
         try {
+            TSocket socket = new TSocket(_host, _port);
+            if(_timeout!=null) {
+                socket.setTimeout(_timeout);
+            }
+
             //locate login configuration 
-            Configuration login_conf = AuthUtils.GetConfiguration(storm_conf);
+            Configuration login_conf = AuthUtils.GetConfiguration(_conf);
 
             //construct a transport plugin
-            ITransportPlugin  transportPlugin = AuthUtils.GetTransportPlugin(storm_conf, login_conf);
+            ITransportPlugin transportPlugin = AuthUtils.GetTransportPlugin(_type, _conf, login_conf);
 
-            //create a socket with server
-            if(host==null) {
-                throw new IllegalArgumentException("host is not set");
-            }
-            if(port<=0) {
-                throw new IllegalArgumentException("invalid port: "+port);
-            }            
-            TSocket socket = new TSocket(host, port);
-            if(timeout!=null) {
-                socket.setTimeout(timeout);
-            }
             final TTransport underlyingTransport = socket;
 
+            //TODO get this from type instead of hardcoding to Nimbus.
             //establish client-server transport via plugin
-            _transport =  transportPlugin.connect(underlyingTransport, host); 
+            //do retries if the connect fails
+            TBackoffConnect connectionRetry 
+                = new TBackoffConnect(
+                                      Utils.getInt(_conf.get(Config.STORM_NIMBUS_RETRY_TIMES)),
+                                      Utils.getInt(_conf.get(Config.STORM_NIMBUS_RETRY_INTERVAL)),
+                                      Utils.getInt(_conf.get(Config.STORM_NIMBUS_RETRY_INTERVAL_CEILING)));
+            _transport = connectionRetry.doConnectWithRetry(transportPlugin, underlyingTransport, _host);
         } catch (IOException ex) {
             throw new RuntimeException(ex);
         }
         _protocol = null;
-        if (_transport != null)
+        if (_transport != null) {
             _protocol = new  TBinaryProtocol(_transport);
+        }
     }
 
-    public TTransport transport() {
-        return _transport;
-    }
-
-    public void close() {
-        _transport.close();
+    public synchronized void close() {
+        if (_transport != null) {
+            _transport.close();
+            _transport = null;
+            _protocol = null;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/ThriftConnectionType.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/ThriftConnectionType.java b/storm-core/src/jvm/backtype/storm/security/auth/ThriftConnectionType.java
new file mode 100644
index 0000000..f9be7ae
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/security/auth/ThriftConnectionType.java
@@ -0,0 +1,77 @@
+/**
+ * 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.security.auth;
+
+import backtype.storm.utils.Utils;
+import backtype.storm.Config;
+
+import java.util.Map;
+
+/**
+ * The purpose for which the Thrift server is created.
+ */
+public enum ThriftConnectionType {
+    NIMBUS(Config.NIMBUS_THRIFT_TRANSPORT_PLUGIN, Config.NIMBUS_THRIFT_PORT, null,
+         Config.NIMBUS_THRIFT_THREADS, Config.NIMBUS_THRIFT_MAX_BUFFER_SIZE),
+    DRPC(Config.DRPC_THRIFT_TRANSPORT_PLUGIN, Config.DRPC_PORT, Config.DRPC_QUEUE_SIZE,
+         Config.DRPC_WORKER_THREADS, Config.DRPC_MAX_BUFFER_SIZE),
+    DRPC_INVOCATIONS(Config.DRPC_INVOCATIONS_THRIFT_TRANSPORT_PLUGIN, Config.DRPC_INVOCATIONS_PORT, null,
+         Config.DRPC_INVOCATIONS_THREADS, Config.DRPC_MAX_BUFFER_SIZE);
+
+    private final String _transConf;
+    private final String _portConf;
+    private final String _qConf;
+    private final String _threadsConf;
+    private final String _buffConf;
+
+    ThriftConnectionType(String transConf, String portConf, String qConf,
+                         String threadsConf, String buffConf) {
+        _transConf = transConf;
+        _portConf = portConf;
+        _qConf = qConf;
+        _threadsConf = threadsConf;
+        _buffConf = buffConf;
+    }
+
+    public String getTransportPlugin(Map conf) {
+        String ret = (String)conf.get(_transConf);
+        if (ret == null) {
+            ret = (String)conf.get(Config.STORM_THRIFT_TRANSPORT_PLUGIN);
+        }
+        return ret;
+    }
+
+    public int getPort(Map conf) {
+        return Utils.getInt(conf.get(_portConf));
+    }
+
+    public Integer getQueueSize(Map conf) {
+        if (_qConf == null) {
+            return null;
+        }
+        return (Integer)conf.get(_qConf);
+    }
+
+    public int getNumThreads(Map conf) { 
+        return Utils.getInt(conf.get(_threadsConf));
+    }
+
+    public int getMaxBufferSize(Map conf) {
+        return Utils.getInt(conf.get(_buffConf));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/ThriftServer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/ThriftServer.java b/storm-core/src/jvm/backtype/storm/security/auth/ThriftServer.java
index ec6e4ad..64243ce 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/ThriftServer.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/ThriftServer.java
@@ -18,27 +18,28 @@
 package backtype.storm.security.auth;
 
 import java.util.Map;
+
 import javax.security.auth.login.Configuration;
+
 import org.apache.thrift.TProcessor;
 import org.apache.thrift.server.TServer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import backtype.storm.utils.Utils;
 
 public class ThriftServer {
     private static final Logger LOG = LoggerFactory.getLogger(ThriftServer.class);
     private Map _storm_conf; //storm configuration
     protected TProcessor _processor = null;
-    private int _port = 0;
+    private final ThriftConnectionType _type;
     private TServer _server = null;
     private Configuration _login_conf;
     
-    public ThriftServer(Map storm_conf, TProcessor processor, int port) {
+    public ThriftServer(Map storm_conf, TProcessor processor, ThriftConnectionType type) {
+        _storm_conf = storm_conf;
+        _processor = processor;
+        _type = type;
+
         try {
-            _storm_conf = storm_conf;
-            _processor = processor;
-            _port = port;
-            
             //retrieve authentication configuration 
             _login_conf = AuthUtils.GetConfiguration(_storm_conf);
         } catch (Exception x) {
@@ -63,10 +64,10 @@ public class ThriftServer {
     public void serve()  {
         try {
             //locate our thrift transport plugin
-            ITransportPlugin  transportPlugin = AuthUtils.GetTransportPlugin(_storm_conf, _login_conf);
+            ITransportPlugin  transportPlugin = AuthUtils.GetTransportPlugin(_type, _storm_conf, _login_conf);
 
             //server
-            _server = transportPlugin.getServer(_port, _processor);
+            _server = transportPlugin.getServer(_processor);
 
             //start accepting requests
             _server.serve();

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DRPCAuthorizerBase.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DRPCAuthorizerBase.java b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DRPCAuthorizerBase.java
new file mode 100644
index 0000000..8951edd
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DRPCAuthorizerBase.java
@@ -0,0 +1,46 @@
+package backtype.storm.security.auth.authorizer;
+
+import java.util.Map;
+
+import backtype.storm.security.auth.IAuthorizer;
+import backtype.storm.security.auth.ReqContext;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class DRPCAuthorizerBase implements IAuthorizer {
+    public static Logger LOG = LoggerFactory.getLogger(DRPCAuthorizerBase.class);
+
+    /**
+     * A key name for the function requested to be executed by a user.
+     */
+    public static final String FUNCTION_NAME = "function.name";
+
+    @Override
+    public abstract void prepare(Map conf);
+
+    abstract protected boolean permitClientRequest(ReqContext context, String operation, Map params);
+
+    abstract protected boolean permitInvocationRequest(ReqContext context, String operation, Map params);
+    
+    /**
+     * Authorizes request from to the DRPC server.
+     * @param context the client request context
+     * @param operation the operation requested by the DRPC server
+     * @param params a Map with any key-value entries of use to the authorization implementation
+     */
+    @Override
+    public boolean permit(ReqContext context, String operation, Map params) {
+        if ("execute".equals(operation)) {
+            return permitClientRequest(context, operation, params);
+        } else if ("failRequest".equals(operation) || 
+                "fetchRequest".equals(operation) || 
+                "result".equals(operation)) {
+            return permitInvocationRequest(context, operation, params);
+        }
+        // Deny unsupported operations.
+        LOG.warn("Denying unsupported operation \""+operation+"\" from "+
+                context.remoteAddress());
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer.java b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer.java
new file mode 100644
index 0000000..45eaea5
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer.java
@@ -0,0 +1,157 @@
+package backtype.storm.security.auth.authorizer;
+
+import java.lang.reflect.Field;
+import java.security.Principal;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import backtype.storm.Config;
+import backtype.storm.security.auth.ReqContext;
+import backtype.storm.security.auth.authorizer.DRPCAuthorizerBase;
+import backtype.storm.security.auth.AuthUtils;
+import backtype.storm.security.auth.IPrincipalToLocal;
+import backtype.storm.utils.Utils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DRPCSimpleACLAuthorizer extends DRPCAuthorizerBase {
+    public static Logger LOG =
+        LoggerFactory.getLogger(DRPCSimpleACLAuthorizer.class);
+
+    public static final String CLIENT_USERS_KEY = "client.users";
+    public static final String INVOCATION_USER_KEY = "invocation.user";
+    public static final String FUNCTION_KEY = "function.name";
+
+    protected String _aclFileName = "";
+    protected IPrincipalToLocal _ptol;
+    protected boolean _permitWhenMissingFunctionEntry = false;
+
+    protected class AclFunctionEntry {
+        final public Set<String> clientUsers;
+        final public String invocationUser;
+        public AclFunctionEntry(Collection<String> clientUsers,
+                String invocationUser) {
+            this.clientUsers = (clientUsers != null) ?
+                new HashSet<String>(clientUsers) : new HashSet<String>();
+            this.invocationUser = invocationUser;
+        }
+    }
+
+    private volatile Map<String,AclFunctionEntry> _acl = null;
+    private volatile long _lastUpdate = 0;
+
+    protected Map<String,AclFunctionEntry> readAclFromConfig() {
+        //Thread safety is mostly around _acl.  If _acl needs to be updated it is changed atomically
+        //More then one thread may be trying to update it at a time, but that is OK, because the
+        //change is atomic
+        long now = System.currentTimeMillis();
+        if ((now - 5000) > _lastUpdate || _acl == null) {
+            Map<String,AclFunctionEntry> acl = new HashMap<String,AclFunctionEntry>();
+            Map conf = Utils.findAndReadConfigFile(_aclFileName);
+            if (conf.containsKey(Config.DRPC_AUTHORIZER_ACL)) {
+                Map<String,Map<String,?>> confAcl =
+                    (Map<String,Map<String,?>>)
+                    conf.get(Config.DRPC_AUTHORIZER_ACL);
+
+                for (String function : confAcl.keySet()) {
+                    Map<String,?> val = confAcl.get(function);
+                    Collection<String> clientUsers =
+                        val.containsKey(CLIENT_USERS_KEY) ?
+                        (Collection<String>) val.get(CLIENT_USERS_KEY) : null;
+                    String invocationUser =
+                        val.containsKey(INVOCATION_USER_KEY) ?
+                        (String) val.get(INVOCATION_USER_KEY) : null;
+                    acl.put(function,
+                            new AclFunctionEntry(clientUsers, invocationUser));
+                }
+            } else if (!_permitWhenMissingFunctionEntry) {
+                LOG.warn("Requiring explicit ACL entries, but none given. " +
+                        "Therefore, all operiations will be denied.");
+            }
+            _acl = acl;
+            _lastUpdate = System.currentTimeMillis();
+        }
+        return _acl;
+    }
+
+    @Override
+    public void prepare(Map conf) {
+        Boolean isStrict = 
+                (Boolean) conf.get(Config.DRPC_AUTHORIZER_ACL_STRICT);
+        _permitWhenMissingFunctionEntry = 
+                (isStrict != null && !isStrict) ? true : false;
+        _aclFileName = (String) conf.get(Config.DRPC_AUTHORIZER_ACL_FILENAME);
+        _ptol = AuthUtils.GetPrincipalToLocalPlugin(conf);
+    }
+
+    private String getUserFromContext(ReqContext context) {
+        if (context != null) {
+            Principal princ = context.principal();
+            if (princ != null) {
+                return princ.getName();
+            }
+        }
+        return null;
+    }
+
+    private String getLocalUserFromContext(ReqContext context) {
+        if (context != null) {
+            return _ptol.toLocal(context.principal());
+        }
+        return null;
+    }
+
+    protected boolean permitClientOrInvocationRequest(ReqContext context, Map params,
+            String fieldName) {
+        Map<String,AclFunctionEntry> acl = readAclFromConfig();
+        String function = (String) params.get(FUNCTION_KEY);
+        if (function != null && ! function.isEmpty()) {
+            AclFunctionEntry entry = acl.get(function);
+            if (entry == null && _permitWhenMissingFunctionEntry) {
+                return true;
+            }
+            if (entry != null) {
+                Object value;
+                try {
+                    Field field = AclFunctionEntry.class.getDeclaredField(fieldName);
+                    value = field.get(entry);
+                } catch (Exception ex) {
+                    LOG.warn("Caught Exception while accessing ACL", ex);
+                    return false;
+                }
+                String principal = getUserFromContext(context);
+                String user = getLocalUserFromContext(context);
+                if (value == null) {
+                    LOG.warn("Configuration for function '"+function+"' is "+
+                            "invalid: it should have both an invocation user "+
+                            "and a list of client users defined.");
+                } else if (value instanceof Set && 
+                        (((Set<String>)value).contains(principal) ||
+                        ((Set<String>)value).contains(user))) {
+                    return true;
+                } else if (value instanceof String && 
+                        (value.equals(principal) ||
+                         value.equals(user))) {
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+
+    @Override
+    protected boolean permitClientRequest(ReqContext context, String operation,
+            Map params) {
+        return permitClientOrInvocationRequest(context, params, "clientUsers");
+    }
+
+    @Override
+    protected boolean permitInvocationRequest(ReqContext context, String operation,
+            Map params) {
+        return permitClientOrInvocationRequest(context, params, "invocationUser");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DenyAuthorizer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DenyAuthorizer.java b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DenyAuthorizer.java
index 3af5e3c..5e84b38 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DenyAuthorizer.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DenyAuthorizer.java
@@ -49,9 +49,9 @@ public class DenyAuthorizer implements IAuthorizer {
     public boolean permit(ReqContext context, String operation, Map topology_conf) {
         LOG.info("[req "+ context.requestID()+ "] Access "
                 + " from: " + (context.remoteAddress() == null? "null" : context.remoteAddress().toString())
-                + " principal:"+ (context.principal() == null? "null" : context.principal())
+                + (context.principal() == null? "" : (" principal:"+ context.principal()))
                 +" op:"+operation
-                + " topoology:"+topology_conf.get(Config.TOPOLOGY_NAME));
+                + (topology_conf == null? "" : (" topoology:"+topology_conf.get(Config.TOPOLOGY_NAME))));
         return false;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/authorizer/NoopAuthorizer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/NoopAuthorizer.java b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/NoopAuthorizer.java
index ef95683..9af44d3 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/NoopAuthorizer.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/NoopAuthorizer.java
@@ -49,9 +49,9 @@ public class NoopAuthorizer implements IAuthorizer {
     public boolean permit(ReqContext context, String operation, Map topology_conf) {
         LOG.info("[req "+ context.requestID()+ "] Access "
                 + " from: " + (context.remoteAddress() == null? "null" : context.remoteAddress().toString())
-                + " principal:"+(context.principal() == null? "null" : context.principal()) 
-                +" op:"+ operation
-                + " topoology:"+ topology_conf.get(Config.TOPOLOGY_NAME));
+                + (context.principal() == null? "" : (" principal:"+ context.principal()))
+                +" op:"+operation
+                + (topology_conf == null? "" : (" topoology:"+topology_conf.get(Config.TOPOLOGY_NAME))));
         return true;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java
new file mode 100644
index 0000000..ef13750
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java
@@ -0,0 +1,113 @@
+/**
+ * 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.security.auth.authorizer;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.Collection;
+
+import backtype.storm.Config;
+import backtype.storm.security.auth.IAuthorizer;
+import backtype.storm.security.auth.ReqContext;
+import backtype.storm.security.auth.AuthUtils;
+import backtype.storm.security.auth.IPrincipalToLocal;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An authorization implementation that simply checks if a user is allowed to perform specific
+ * operations.
+ */
+public class SimpleACLAuthorizer implements IAuthorizer {
+    private static final Logger LOG = LoggerFactory.getLogger(SimpleACLAuthorizer.class);
+
+    protected Set<String> _userCommands = new HashSet<String>(Arrays.asList("submitTopology", "fileUpload", "getNimbusConf", "getClusterInfo"));
+    protected Set<String> _supervisorCommands = new HashSet<String>(Arrays.asList("fileDownload"));
+    protected Set<String> _topoCommands = new HashSet<String>(Arrays.asList("killTopology","rebalance","activate","deactivate","getTopologyConf","getTopology","getUserTopology","getTopologyInfo","uploadNewCredentials"));
+
+    protected Set<String> _admins;
+    protected Set<String> _supervisors;
+    protected IPrincipalToLocal _ptol;
+
+    /**
+     * Invoked once immediately after construction
+     * @param conf Storm configuration 
+     */
+    @Override
+    public void prepare(Map conf) {
+        _admins = new HashSet<String>();
+        _supervisors = new HashSet<String>();
+
+        if (conf.containsKey(Config.NIMBUS_ADMINS)) {
+            _admins.addAll((Collection<String>)conf.get(Config.NIMBUS_ADMINS));
+        }
+        if (conf.containsKey(Config.NIMBUS_SUPERVISOR_USERS)) {
+            _supervisors.addAll((Collection<String>)conf.get(Config.NIMBUS_SUPERVISOR_USERS));
+        }
+        _ptol = AuthUtils.GetPrincipalToLocalPlugin(conf);
+    }
+
+    /**
+     * permit() method is invoked for each incoming Thrift request
+     * @param context request context includes info about 
+     * @param operation operation name
+     * @param topology_storm configuration of targeted topology 
+     * @return true if the request is authorized, false if reject
+     */
+    @Override
+    public boolean permit(ReqContext context, String operation, Map topology_conf) {
+
+        LOG.info("[req "+ context.requestID()+ "] Access "
+                 + " from: " + (context.remoteAddress() == null? "null" : context.remoteAddress().toString())
+                 + (context.principal() == null? "" : (" principal:"+ context.principal()))
+                 +" op:"+operation
+                 + (topology_conf == null? "" : (" topoology:"+topology_conf.get(Config.TOPOLOGY_NAME))));
+       
+        String principal = context.principal().getName();
+        String user = _ptol.toLocal(context.principal());
+
+        if (_admins.contains(principal) || _admins.contains(user)) {
+            return true;
+        }
+
+        if (_supervisors.contains(principal) || _supervisors.contains(user)) {
+            return _supervisorCommands.contains(operation);
+        }
+
+        if (_userCommands.contains(operation)) {
+            return true;
+        }
+
+        if (_topoCommands.contains(operation)) {
+            Set topoUsers = new HashSet<String>();
+            if (topology_conf.containsKey(Config.TOPOLOGY_USERS)) {
+                topoUsers.addAll((Collection<String>)topology_conf.get(Config.TOPOLOGY_USERS));
+            }
+
+            if (topoUsers.contains(principal) || topoUsers.contains(user)) {
+                return true;
+            }
+        }
+         
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleWhitelistAuthorizer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleWhitelistAuthorizer.java b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleWhitelistAuthorizer.java
new file mode 100644
index 0000000..55109f9
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleWhitelistAuthorizer.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package backtype.storm.security.auth.authorizer;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.Collection;
+
+import backtype.storm.Config;
+import backtype.storm.security.auth.IAuthorizer;
+import backtype.storm.security.auth.ReqContext;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An authorization implementation that simply checks a whitelist of users that
+ * are allowed to use the cluster.
+ */
+public class SimpleWhitelistAuthorizer implements IAuthorizer {
+    private static final Logger LOG = LoggerFactory.getLogger(SimpleWhitelistAuthorizer.class);
+    public static String WHITELIST_USERS_CONF = "storm.auth.simple-white-list.users";
+    protected Set<String> users;
+
+    /**
+     * Invoked once immediately after construction
+     * @param conf Storm configuration 
+     */
+    @Override
+    public void prepare(Map conf) {
+        users = new HashSet<String>();
+        if (conf.containsKey(WHITELIST_USERS_CONF)) {
+            users.addAll((Collection<String>)conf.get(WHITELIST_USERS_CONF));
+        }
+    }
+
+    /**
+     * permit() method is invoked for each incoming Thrift request
+     * @param context request context includes info about 
+     * @param operation operation name
+     * @param topology_storm configuration of targeted topology 
+     * @return true if the request is authorized, false if reject
+     */
+    @Override
+    public boolean permit(ReqContext context, String operation, Map topology_conf) {
+        LOG.info("[req "+ context.requestID()+ "] Access "
+                 + " from: " + (context.remoteAddress() == null? "null" : context.remoteAddress().toString())
+                 + (context.principal() == null? "" : (" principal:"+ context.principal()))
+                 +" op:"+operation
+                 + (topology_conf == null? "" : (" topoology:"+topology_conf.get(Config.TOPOLOGY_NAME))));
+        return context.principal() != null ? users.contains(context.principal().getName()) : false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/digest/DigestSaslTransportPlugin.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/digest/DigestSaslTransportPlugin.java b/storm-core/src/jvm/backtype/storm/security/auth/digest/DigestSaslTransportPlugin.java
index 4abf98b..cb68579 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/digest/DigestSaslTransportPlugin.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/digest/DigestSaslTransportPlugin.java
@@ -50,6 +50,7 @@ public class DigestSaslTransportPlugin extends SaslTransportPlugin {
         return factory;
     }
 
+    @Override
     public TTransport connect(TTransport transport, String serverHost) throws TTransportException, IOException {
         ClientCallbackHandler client_callback_handler = new ClientCallbackHandler(login_conf);
         TSaslClientTransport wrapper_transport = new TSaslClientTransport(DIGEST, 

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGT.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGT.java b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGT.java
new file mode 100644
index 0000000..52bf540
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGT.java
@@ -0,0 +1,267 @@
+/**
+ * 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.security.auth.kerberos;
+
+import backtype.storm.security.auth.IAutoCredentials;
+import backtype.storm.security.auth.ICredentialsRenewer;
+import backtype.storm.security.auth.AuthUtils;
+
+import java.util.Date;
+import java.util.Map;
+import java.util.Set;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.lang.reflect.Method;
+import java.lang.reflect.Constructor;
+import java.security.Principal;
+import java.util.concurrent.atomic.AtomicReference;
+
+import javax.security.auth.kerberos.KerberosTicket;
+import javax.security.auth.kerberos.KerberosPrincipal;
+import javax.security.auth.login.Configuration;
+import javax.security.auth.login.LoginException;
+import javax.security.auth.login.LoginContext;
+import javax.security.auth.RefreshFailedException;
+import javax.security.auth.Subject;
+import javax.xml.bind.DatatypeConverter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Automatically take a user's TGT, and push it, and renew it in Nimbus.
+ */
+public class AutoTGT implements IAutoCredentials, ICredentialsRenewer {
+    private static final Logger LOG = LoggerFactory.getLogger(AutoTGT.class);
+    private static final float TICKET_RENEW_WINDOW = 0.80f;
+    protected static AtomicReference<KerberosTicket> kerbTicket = new AtomicReference<KerberosTicket>();
+    private Map conf;
+
+    public void prepare(Map conf) {
+        this.conf = conf;
+    }
+
+    private static KerberosTicket getTGT(Subject subject) {
+        Set<KerberosTicket> tickets = subject.getPrivateCredentials(KerberosTicket.class);
+        for(KerberosTicket ticket: tickets) {
+            KerberosPrincipal server = ticket.getServer();
+            if (server.getName().equals("krbtgt/" + server.getRealm() + "@" + server.getRealm())) {
+                tickets = null;
+                return ticket;
+            }
+        }
+        tickets = null;
+        return null;
+    } 
+
+    @Override
+    public void populateCredentials(Map<String, String> credentials) {
+        //Log the user in and get the TGT
+        try {
+            Configuration login_conf = AuthUtils.GetConfiguration(conf);
+            ClientCallbackHandler client_callback_handler = new ClientCallbackHandler(login_conf);
+        
+            //login our user
+            Configuration.setConfiguration(login_conf); 
+            LoginContext lc = new LoginContext(AuthUtils.LOGIN_CONTEXT_CLIENT, client_callback_handler);
+            try {
+                lc.login();
+                final Subject subject = lc.getSubject();
+                KerberosTicket tgt = getTGT(subject);
+
+                if (tgt == null) { //error
+                    throw new RuntimeException("Fail to verify user principal with section \""
+                            +AuthUtils.LOGIN_CONTEXT_CLIENT+"\" in login configuration file "+ login_conf);
+                }
+
+                if (!tgt.isForwardable()) {
+                    throw new RuntimeException("The TGT found is not forwardable");
+                }
+
+                if (!tgt.isRenewable()) {
+                    throw new RuntimeException("The TGT found is not renewable");
+                }
+
+                LOG.info("Pushing TGT for "+tgt.getClient()+" to topology.");
+                saveTGT(tgt, credentials);
+            } finally {
+                lc.logout();
+            }
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static void saveTGT(KerberosTicket tgt, Map<String, String> credentials) {
+        try {
+            ByteArrayOutputStream bao = new ByteArrayOutputStream();
+            ObjectOutputStream out = new ObjectOutputStream(bao);
+            out.writeObject(tgt);
+            out.flush();
+            out.close();
+            credentials.put("TGT", DatatypeConverter.printBase64Binary(bao.toByteArray()));
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static KerberosTicket getTGT(Map<String, String> credentials) {
+        KerberosTicket ret = null;
+        if (credentials != null && credentials.containsKey("TGT")) {
+            try {
+                ByteArrayInputStream bin = new ByteArrayInputStream(DatatypeConverter.parseBase64Binary(credentials.get("TGT")));
+                ObjectInputStream in = new ObjectInputStream(bin);
+                ret = (KerberosTicket)in.readObject();
+                in.close();
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        }
+        return ret;
+    }
+
+    @Override
+    public void updateSubject(Subject subject, Map<String, String> credentials) {
+        populateSubjectWithTGT(subject, credentials);
+    }
+
+    @Override
+    public void populateSubject(Subject subject, Map<String, String> credentials) {
+        populateSubjectWithTGT(subject, credentials);
+        loginHadoopUser(subject);
+    }
+
+    private void populateSubjectWithTGT(Subject subject, Map<String, String> credentials) {
+        KerberosTicket tgt = getTGT(credentials);
+        if (tgt != null) {
+            KerberosTicket oldTGT = getTGT(subject);
+            subject.getPrivateCredentials().add(tgt);
+            if (oldTGT != null && !oldTGT.equals(tgt)) {
+                subject.getPrivateCredentials().remove(oldTGT);
+            }
+            subject.getPrincipals().add(tgt.getClient());
+            kerbTicket.set(tgt);
+        } else {
+            LOG.info("No TGT found in credentials");
+        }
+    }
+
+    /**
+     * Hadoop does not just go off of a TGT, it needs a bit more.  This
+     * should fill in the rest.
+     * @param subject the subject that should have a TGT in it.
+     */
+    private void loginHadoopUser(Subject subject) {
+        Class<?> ugi = null;
+        try {
+            ugi = Class.forName("org.apache.hadoop.security.UserGroupInformation");
+        } catch (ClassNotFoundException e) {
+            LOG.info("Hadoop was not found on the class path");
+            return;
+        }
+        try {
+            Method isSecEnabled = ugi.getMethod("isSecurityEnabled");
+            if (!((Boolean)isSecEnabled.invoke(null))) {
+                LOG.warn("Hadoop is on the classpath but not configured for " +
+                  "security, if you want security you need to be sure that " +
+                  "hadoop.security.authentication=kerberos in core-site.xml " +
+                  "in your jar");
+                return;
+            }
+ 
+            try {
+                Method login = ugi.getMethod("loginUserFromSubject", Subject.class);
+                login.invoke(null, subject);
+            } catch (NoSuchMethodException me) {
+                //The version of Hadoop does not have the needed client changes.
+                // So don't look now, but do something really ugly to work around it.
+                // This is because we are reaching into the hidden bits of Hadoop security, and it works for now, but may stop at any point in time.
+
+                //We are just trying to do the following
+                // Configuration conf = new Configuration();
+                // HadoopKerberosName.setConfiguration(conf);
+                // subject.getPrincipals().add(new User(tgt.getClient().toString(), AuthenticationMethod.KERBEROS, null));
+                String name = getTGT(subject).getClient().toString();
+
+                LOG.warn("The Hadoop client does not have loginUserFromSubject, Trying to hack around it. This may not work...");
+                Class<?> confClass = Class.forName("org.apache.hadoop.conf.Configuration");
+                Constructor confCons = confClass.getConstructor();
+                Object conf = confCons.newInstance();
+                Class<?> hknClass = Class.forName("org.apache.hadoop.security.HadoopKerberosName");
+                Method hknSetConf = hknClass.getMethod("setConfiguration",confClass);
+                hknSetConf.invoke(null, conf);
+
+                Class<?> authMethodClass = Class.forName("org.apache.hadoop.security.UserGroupInformation$AuthenticationMethod");
+                Object kerbAuthMethod = null;
+                for (Object authMethod : authMethodClass.getEnumConstants()) {
+                    if ("KERBEROS".equals(authMethod.toString())) {
+                        kerbAuthMethod = authMethod;
+                        break;
+                    }
+                }
+
+                Class<?> userClass = Class.forName("org.apache.hadoop.security.User");
+                Constructor userCons = userClass.getConstructor(String.class, authMethodClass, LoginContext.class);
+                userCons.setAccessible(true);
+                Object user = userCons.newInstance(name, kerbAuthMethod, null);
+                subject.getPrincipals().add((Principal)user);
+            }
+        } catch (Exception e) {
+            LOG.warn("Something went wrong while trying to initialize Hadoop through reflection. This version of hadoop may not be compatible.", e);
+        }
+    }
+
+    private long getRefreshTime(KerberosTicket tgt) {
+        long start = tgt.getStartTime().getTime();
+        long end = tgt.getEndTime().getTime();
+        return start + (long) ((end - start) * TICKET_RENEW_WINDOW);
+    }
+
+    @Override
+    public void renew(Map<String,String> credentials) {
+        KerberosTicket tgt = getTGT(credentials);
+        if (tgt != null) {
+            long refreshTime = getRefreshTime(tgt);
+            long now = System.currentTimeMillis();
+            if (now >= refreshTime) {
+                try {
+                    LOG.info("Renewing TGT for "+tgt.getClient());
+                    tgt.refresh();
+                    saveTGT(tgt, credentials);
+                } catch (RefreshFailedException e) {
+                    LOG.warn("Failed to refresh TGT", e);
+                }
+            }
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        AutoTGT at = new AutoTGT();
+        Map conf = new java.util.HashMap();
+        conf.put("java.security.auth.login.config", args[0]);
+        at.prepare(conf);
+        Map<String,String> creds = new java.util.HashMap<String,String>();
+        at.populateCredentials(creds);
+        Subject s = new Subject();
+        at.populateSubject(s, creds);
+        LOG.info("Got a Subject "+s);
+    }
+}


[06/14] STORM-216: Added Authentication and Authorization.

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGTKrb5LoginModule.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGTKrb5LoginModule.java b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGTKrb5LoginModule.java
new file mode 100644
index 0000000..807abe3
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGTKrb5LoginModule.java
@@ -0,0 +1,108 @@
+/**
+ * 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.security.auth.kerberos;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.security.Principal;
+import java.util.Map;
+import javax.security.auth.Subject;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.kerberos.KerberosTicket;
+import javax.security.auth.login.LoginException;
+import javax.security.auth.spi.LoginModule;
+
+
+/**
+ * Custom LoginModule to enable Auto Login based on cached ticket
+ */
+public class AutoTGTKrb5LoginModule implements LoginModule {
+    private static final Logger LOG = LoggerFactory.getLogger(AutoTGTKrb5LoginModule.class);
+
+    // initial state
+    private Subject subject;
+
+    protected KerberosTicket kerbTicket = null;
+
+    public void initialize(Subject subject,
+                           CallbackHandler callbackHandler,
+                           Map<String, ?> sharedState,
+                           Map<String, ?> options) {
+
+        this.subject = subject;
+    }
+
+    public boolean login() throws LoginException {
+        LOG.debug("Acquire TGT from Cache");
+        getKerbTicketFromCache();
+        if (kerbTicket != null) {
+            return true;
+        } else {
+            throw new LoginException("Authentication failed, the TGT not found.");
+        }
+    }
+
+    protected void getKerbTicketFromCache() {
+        kerbTicket = AutoTGT.kerbTicket.get();
+    }
+
+    protected Principal getKerbTicketClient() {
+        if (kerbTicket != null) {
+            return kerbTicket.getClient();
+        }
+        return null;
+    }
+
+    public boolean commit() throws LoginException {
+        if (isSucceeded() == false) {
+            return false;
+        }
+        if (subject == null || subject.isReadOnly()) {
+            kerbTicket = null;
+            throw new LoginException("Authentication failed because the Subject is invalid.");
+        }
+        // Let us add the kerbClientPrinc and kerbTicket
+        subject.getPrivateCredentials().add(kerbTicket);
+        subject.getPrincipals().add(getKerbTicketClient());
+        LOG.debug("Commit Succeeded.");
+        return true;
+    }
+
+    public boolean abort() throws LoginException {
+        if (isSucceeded() == false) {
+            return false;
+        } else {
+            return logout();
+        }
+    }
+
+    public boolean logout() throws LoginException {
+        if (subject != null && !subject.isReadOnly() && kerbTicket != null) {
+            subject.getPrincipals().remove(kerbTicket.getClient());
+            subject.getPrivateCredentials().remove(kerbTicket);
+        }
+        kerbTicket = null;
+        return true;
+    }
+
+    private boolean isSucceeded() {
+        return kerbTicket != null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGTKrb5LoginModuleTest.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGTKrb5LoginModuleTest.java b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGTKrb5LoginModuleTest.java
new file mode 100644
index 0000000..ba34fc9
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGTKrb5LoginModuleTest.java
@@ -0,0 +1,44 @@
+/**
+ * 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.security.auth.kerberos;
+
+import java.security.Principal;
+import javax.security.auth.kerberos.KerberosTicket;
+
+/**
+ * Custom LoginModule extended for testing.
+ */
+public class AutoTGTKrb5LoginModuleTest extends AutoTGTKrb5LoginModule {
+
+    public Principal client = null;
+
+    public void setKerbTicket(KerberosTicket ticket) {
+        this.kerbTicket = ticket;
+    }
+    
+    @Override
+    protected void getKerbTicketFromCache() {
+        // Do nothing.
+    }
+
+    @Override
+    protected Principal getKerbTicketClient() {
+        return this.client;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/kerberos/ClientCallbackHandler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/ClientCallbackHandler.java b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/ClientCallbackHandler.java
new file mode 100644
index 0000000..d46aa8b
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/ClientCallbackHandler.java
@@ -0,0 +1,104 @@
+/**
+ * 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.security.auth.kerberos;
+
+import java.io.IOException;
+import java.util.Map;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.RealmCallback;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import backtype.storm.security.auth.AuthUtils;
+
+/**
+ * SASL client side callback handler.
+ */
+public class ClientCallbackHandler implements CallbackHandler {
+    private static final Logger LOG = LoggerFactory.getLogger(ClientCallbackHandler.class);
+
+    /**
+     * Constructor based on a JAAS configuration
+     * 
+     * For digest, you should have a pair of user name and password defined in this figgure.
+     * 
+     * @param configuration
+     * @throws IOException
+     */
+    public ClientCallbackHandler(Configuration configuration) throws IOException {
+        if (configuration == null) return;
+        AppConfigurationEntry configurationEntries[] = configuration.getAppConfigurationEntry(AuthUtils.LOGIN_CONTEXT_CLIENT);
+        if (configurationEntries == null) {
+            String errorMessage = "Could not find a '"+AuthUtils.LOGIN_CONTEXT_CLIENT
+                    + "' entry in this configuration: Client cannot start.";
+            LOG.error(errorMessage);
+            throw new IOException(errorMessage);
+        }
+    }
+
+    /**
+     * This method is invoked by SASL for authentication challenges
+     * @param callbacks a collection of challenge callbacks 
+     */
+    public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+        for (Callback c : callbacks) {
+            if (c instanceof NameCallback) {
+                LOG.debug("name callback");
+            } else if (c instanceof PasswordCallback) {
+                LOG.debug("password callback");
+                LOG.warn("Could not login: the client is being asked for a password, but the " +
+                        " client code does not currently support obtaining a password from the user." +
+                        " Make sure that the client is configured to use a ticket cache (using" +
+                        " the JAAS configuration setting 'useTicketCache=true)' and restart the client. If" +
+                        " you still get this message after that, the TGT in the ticket cache has expired and must" +
+                        " be manually refreshed. To do so, first determine if you are using a password or a" +
+                        " keytab. If the former, run kinit in a Unix shell in the environment of the user who" +
+                        " is running this client using the command" +
+                        " 'kinit <princ>' (where <princ> is the name of the client's Kerberos principal)." +
+                        " If the latter, do" +
+                        " 'kinit -k -t <keytab> <princ>' (where <princ> is the name of the Kerberos principal, and" +
+                        " <keytab> is the location of the keytab file). After manually refreshing your cache," +
+                        " restart this client. If you continue to see this message after manually refreshing" +
+                        " your cache, ensure that your KDC host's clock is in sync with this host's clock.");
+            } else if (c instanceof AuthorizeCallback) {
+                LOG.debug("authorization callback");
+                AuthorizeCallback ac = (AuthorizeCallback) c;
+                String authid = ac.getAuthenticationID();
+                String authzid = ac.getAuthorizationID();
+                if (authid.equals(authzid)) {
+                    ac.setAuthorized(true);
+                } else {
+                    ac.setAuthorized(false);
+                }
+                if (ac.isAuthorized()) {
+                    ac.setAuthorizedID(authzid);
+                }
+            }  else {
+                throw new UnsupportedCallbackException(c);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java
new file mode 100644
index 0000000..451f87b
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java
@@ -0,0 +1,206 @@
+/**
+ * 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.security.auth.kerberos;
+
+import java.io.IOException;
+import java.security.Principal;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import javax.security.auth.Subject;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.kerberos.KerberosTicket;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
+import javax.security.auth.login.LoginException;
+import javax.security.sasl.Sasl;
+import org.apache.thrift.transport.TSaslClientTransport;
+import org.apache.thrift.transport.TSaslServerTransport;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.thrift.transport.TTransportFactory;
+import org.apache.zookeeper.Login;
+import org.apache.zookeeper.server.auth.KerberosName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import backtype.storm.security.auth.AuthUtils;
+import backtype.storm.security.auth.SaslTransportPlugin;
+
+public class KerberosSaslTransportPlugin extends SaslTransportPlugin {
+    public static final String KERBEROS = "GSSAPI"; 
+    private static final Logger LOG = LoggerFactory.getLogger(KerberosSaslTransportPlugin.class);
+
+    public TTransportFactory getServerTransportFactory() throws IOException {
+        //create an authentication callback handler
+        CallbackHandler server_callback_handler = new ServerCallbackHandler(login_conf);
+        
+        //login our principal
+        Subject subject = null;
+        try {
+            //specify a configuration object to be used
+            Configuration.setConfiguration(login_conf); 
+            //now login
+            Login login = new Login(AuthUtils.LOGIN_CONTEXT_SERVER, server_callback_handler);
+            subject = login.getSubject();
+        } catch (LoginException ex) {
+            LOG.error("Server failed to login in principal:" + ex, ex);
+            throw new RuntimeException(ex);
+        }
+
+        //check the credential of our principal
+        if (subject.getPrivateCredentials(KerberosTicket.class).isEmpty()) { 
+            throw new RuntimeException("Fail to verify user principal with section \""
+                    +AuthUtils.LOGIN_CONTEXT_SERVER+"\" in login configuration file "+ login_conf);
+        }
+
+        String principal = AuthUtils.get(login_conf, AuthUtils.LOGIN_CONTEXT_SERVER, "principal"); 
+        LOG.debug("principal:"+principal);  
+        KerberosName serviceKerberosName = new KerberosName(principal);
+        String serviceName = serviceKerberosName.getServiceName();
+        String hostName = serviceKerberosName.getHostName();
+        Map<String, String> props = new TreeMap<String,String>();
+        props.put(Sasl.QOP, "auth");
+        props.put(Sasl.SERVER_AUTH, "false");
+
+        //create a transport factory that will invoke our auth callback for digest
+        TSaslServerTransport.Factory factory = new TSaslServerTransport.Factory();
+        factory.addServerDefinition(KERBEROS, serviceName, hostName, props, server_callback_handler);
+
+        //create a wrap transport factory so that we could apply user credential during connections
+        TUGIAssumingTransportFactory wrapFactory = new TUGIAssumingTransportFactory(factory, subject); 
+
+        LOG.info("SASL GSSAPI transport factory will be used");
+        return wrapFactory;
+    }
+
+    public TTransport connect(TTransport transport, String serverHost) throws TTransportException, IOException {
+        //create an authentication callback handler
+        ClientCallbackHandler client_callback_handler = new ClientCallbackHandler(login_conf);
+        
+        //login our user
+        Login login = null;
+        try { 
+            //specify a configuration object to be used
+            Configuration.setConfiguration(login_conf); 
+            //now login
+            login  = new Login(AuthUtils.LOGIN_CONTEXT_CLIENT, client_callback_handler);
+        } catch (LoginException ex) {
+            LOG.error("Server failed to login in principal:" + ex, ex);
+            throw new RuntimeException(ex);
+        }
+
+        final Subject subject = login.getSubject();
+        if (subject.getPrivateCredentials(KerberosTicket.class).isEmpty()) { //error
+            throw new RuntimeException("Fail to verify user principal with section \""
+                        +AuthUtils.LOGIN_CONTEXT_CLIENT+"\" in login configuration file "+ login_conf);
+        }
+
+        final String principal = getPrincipal(subject); 
+        String serviceName = AuthUtils.get(login_conf, AuthUtils.LOGIN_CONTEXT_CLIENT, "serviceName");
+        if (serviceName == null) {
+            serviceName = AuthUtils.SERVICE; 
+        }
+        Map<String, String> props = new TreeMap<String,String>();
+        props.put(Sasl.QOP, "auth");
+        props.put(Sasl.SERVER_AUTH, "false");
+
+        LOG.debug("SASL GSSAPI client transport is being established");
+        final TTransport sasalTransport = new TSaslClientTransport(KERBEROS, 
+                principal, 
+                serviceName, 
+                serverHost,
+                props,
+                null, 
+                transport);
+
+        //open Sasl transport with the login credential
+        try {
+            Subject.doAs(subject,
+                    new PrivilegedExceptionAction<Void>() {
+                public Void run() {
+                    try {
+                        LOG.debug("do as:"+ principal);
+                        sasalTransport.open();
+                    }
+                    catch (Exception e) {
+                        LOG.error("Client failed to open SaslClientTransport to interact with a server during session initiation: " + e, e);
+                    }
+                    return null;
+                }
+            });
+        } catch (PrivilegedActionException e) {
+            throw new RuntimeException(e);
+        }
+
+        return sasalTransport;
+    }
+
+    private String getPrincipal(Subject subject) {
+        Set<Principal> principals = (Set<Principal>)subject.getPrincipals();
+        if (principals==null || principals.size()<1) {
+            LOG.info("No principal found in login subject");
+            return null;
+        }
+        return ((Principal)(principals.toArray()[0])).getName();
+    }
+
+    /** A TransportFactory that wraps another one, but assumes a specified UGI
+     * before calling through.                                                                                                                                                      
+     *                                                                                                                                                                              
+     * This is used on the server side to assume the server's Principal when accepting                                                                                              
+     * clients.                                                                                                                                                                     
+     */
+    static class TUGIAssumingTransportFactory extends TTransportFactory {
+        private final Subject subject;
+        private final TTransportFactory wrapped;
+
+        public TUGIAssumingTransportFactory(TTransportFactory wrapped, Subject subject) {
+            this.wrapped = wrapped;
+            this.subject = subject;
+
+            Set<Principal> principals = (Set<Principal>)subject.getPrincipals();
+            if (principals.size()>0) 
+                LOG.info("Service principal:"+ ((Principal)(principals.toArray()[0])).getName());
+        }
+
+        @Override
+        public TTransport getTransport(final TTransport trans) {
+            try {
+                return Subject.doAs(subject,
+                        new PrivilegedExceptionAction<TTransport>() {
+                    public TTransport run() {
+                        try {
+                            return wrapped.getTransport(trans);
+                        }
+                        catch (Exception e) {
+                            LOG.error("Storm server failed to open transport to interact with a client during session initiation: " + e, e);
+                            return null;
+                        }
+                    }
+                });
+            } catch (PrivilegedActionException e) {
+                LOG.error("Storm server experienced a PrivilegedActionException exception while creating a transport using a JAAS principal context:" + e, e);
+                return null;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/kerberos/ServerCallbackHandler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/ServerCallbackHandler.java b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/ServerCallbackHandler.java
new file mode 100644
index 0000000..9dc75c4
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/ServerCallbackHandler.java
@@ -0,0 +1,86 @@
+/**
+ * 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.security.auth.kerberos;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.RealmCallback;
+
+import backtype.storm.security.auth.AuthUtils;
+
+/**
+ * SASL server side callback handler
+ */
+public class ServerCallbackHandler implements CallbackHandler {
+    private static final Logger LOG = LoggerFactory.getLogger(ServerCallbackHandler.class);
+
+    private String userName;
+
+    public ServerCallbackHandler(Configuration configuration) throws IOException {
+        if (configuration==null) return;
+
+        AppConfigurationEntry configurationEntries[] = configuration.getAppConfigurationEntry(AuthUtils.LOGIN_CONTEXT_SERVER);
+        if (configurationEntries == null) {
+            String errorMessage = "Could not find a '"+AuthUtils.LOGIN_CONTEXT_SERVER+"' entry in this configuration: Server cannot start.";
+            LOG.error(errorMessage);
+            throw new IOException(errorMessage);
+        }
+    }
+
+    public void handle(Callback[] callbacks) throws UnsupportedCallbackException {
+        for (Callback callback : callbacks) {
+            if (callback instanceof NameCallback) {
+                handleNameCallback((NameCallback) callback);
+            } else if (callback instanceof PasswordCallback) {
+                handlePasswordCallback((PasswordCallback) callback);
+            } else if (callback instanceof AuthorizeCallback) {
+                handleAuthorizeCallback((AuthorizeCallback) callback);
+            }
+        }
+    }
+
+    private void handleNameCallback(NameCallback nc) {
+        LOG.debug("handleNameCallback");
+        userName = nc.getDefaultName();
+        nc.setName(nc.getDefaultName());
+    }
+
+    private void handlePasswordCallback(PasswordCallback pc) {
+        LOG.warn("No password found for user: " + userName);
+    }
+
+    private void handleAuthorizeCallback(AuthorizeCallback ac) {
+        String authenticationID = ac.getAuthenticationID();
+        LOG.debug("Successfully authenticated client: authenticationID=" + authenticationID);
+        ac.setAuthorized(true);
+
+        ac.setAuthorizedID(authenticationID);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/kerberos/jaas_kerberos_cluster.conf
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/jaas_kerberos_cluster.conf b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/jaas_kerberos_cluster.conf
new file mode 100644
index 0000000..92a1399
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/jaas_kerberos_cluster.conf
@@ -0,0 +1,31 @@
+/* 
+This is a sample JAAS configuration for Storm servers to handle Kerberos authentication
+*/
+
+/*
+ StormServer section should contains the info about server keytab file and server principal.
+ In Storm, we have 2 thrift servers: Nimbus and DRPC. These servers could be assigned with
+ different principals.
+*/
+StormServer {
+       com.sun.security.auth.module.Krb5LoginModule required
+       useKeyTab=true
+       keyTab="/etc/storm_server.keytab"
+       storeKey=true
+       useTicketCache=false
+       principal="storm_service/carcloth.corp.acme.com@STORM.CORP.ACME.COM";
+};
+
+/*
+StormClient section should contains the info about client keytab file and client principal. 
+For example, Supervisors are clients of Nimbus, and we should assign keytab/principal for supervisors.
+*/
+StormClient {
+       com.sun.security.auth.module.Krb5LoginModule required
+       useKeyTab=true
+       keyTab="/etc/storm_client.keytab"
+       storeKey=true
+       useTicketCache=false
+       serviceName="storm_service";
+};
+

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/security/auth/kerberos/jaas_kerberos_launcher.conf
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/jaas_kerberos_launcher.conf b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/jaas_kerberos_launcher.conf
new file mode 100644
index 0000000..138e1f3
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/jaas_kerberos_launcher.conf
@@ -0,0 +1,12 @@
+/*
+ This is a sample JAAS configuration for Storm topology launcher/submitter.
+ Since launcher machines are typically accessible by many folks, we 
+ encourage you to leverage "kinit", instead of keytab.  
+*/
+StormClient {
+       com.sun.security.auth.module.Krb5LoginModule required
+       doNotPrompt=true
+       useTicketCache=true
+       serviceName="storm_service";
+};
+

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/testing/SingleUserSimpleTransport.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/testing/SingleUserSimpleTransport.java b/storm-core/src/jvm/backtype/storm/testing/SingleUserSimpleTransport.java
new file mode 100644
index 0000000..4d25ac7
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/testing/SingleUserSimpleTransport.java
@@ -0,0 +1,37 @@
+/**
+ * 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.testing;
+
+import backtype.storm.security.auth.SimpleTransportPlugin;
+import javax.security.auth.Subject;
+import java.security.Principal;
+import java.util.HashSet;
+
+
+public class SingleUserSimpleTransport extends SimpleTransportPlugin {
+   @Override
+   protected Subject getDefaultSubject() {
+       HashSet<Principal> principals = new HashSet<Principal>();
+       principals.add(new Principal() {
+          public String getName() { return "user"; }
+          public String toString() { return "user"; }
+       });
+       return new Subject(true, principals, new HashSet<Object>(), new HashSet<Object>());
+   } 
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/transactional/state/TestTransactionalState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/transactional/state/TestTransactionalState.java b/storm-core/src/jvm/backtype/storm/transactional/state/TestTransactionalState.java
new file mode 100644
index 0000000..3d4a463
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/transactional/state/TestTransactionalState.java
@@ -0,0 +1,47 @@
+/**
+ * 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.transactional.state;
+
+import java.util.List;
+import java.util.Map;
+
+import backtype.storm.utils.ZookeeperAuthInfo;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.data.ACL;
+
+/**
+ * Facilitates testing of non-public methods in the parent class.
+ */
+public class TestTransactionalState extends TransactionalState {
+
+    /**
+     * Matching constructor in absence of a default constructor in the parent
+     * class.
+     */
+    protected TestTransactionalState(Map conf, String id, Map componentConf, String subroot) {
+        super(conf, id, componentConf, subroot);
+    }
+
+    public static void createNode(CuratorFramework curator, 
+            String rootDir, byte[] data, List<ACL> acls, CreateMode mode)
+            throws Exception {
+       TransactionalState.createNode(curator, rootDir, data, acls, mode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/transactional/state/TransactionalState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/transactional/state/TransactionalState.java b/storm-core/src/jvm/backtype/storm/transactional/state/TransactionalState.java
index 91fc666..5afcd0a 100644
--- a/storm-core/src/jvm/backtype/storm/transactional/state/TransactionalState.java
+++ b/storm-core/src/jvm/backtype/storm/transactional/state/TransactionalState.java
@@ -21,18 +21,25 @@ import backtype.storm.Config;
 import backtype.storm.serialization.KryoValuesDeserializer;
 import backtype.storm.serialization.KryoValuesSerializer;
 import backtype.storm.utils.Utils;
+import backtype.storm.utils.ZookeeperAuthInfo;
 import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.api.ProtectACLCreateModePathAndBytesable;
+import org.apache.curator.framework.api.PathAndBytesable;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
 
 public class TransactionalState {
     CuratorFramework _curator;
     KryoValuesSerializer _ser;
     KryoValuesDeserializer _des;
+    List<ACL> _zkAcls = null;
     
     public static TransactionalState newUserState(Map conf, String id, Map componentConf) {
         return new TransactionalState(conf, id, componentConf, "user");
@@ -51,26 +58,55 @@ public class TransactionalState {
                          componentConf
                               .get(Config.TOPOLOGY_KRYO_REGISTER));
             }
-            String rootDir = conf.get(Config.TRANSACTIONAL_ZOOKEEPER_ROOT) + "/" + id + "/" + subroot;
+            String transactionalRoot = (String)conf.get(Config.TRANSACTIONAL_ZOOKEEPER_ROOT);
+            String rootDir = transactionalRoot + "/" + id + "/" + subroot;
             List<String> servers = (List<String>) getWithBackup(conf, Config.TRANSACTIONAL_ZOOKEEPER_SERVERS, Config.STORM_ZOOKEEPER_SERVERS);
             Object port = getWithBackup(conf, Config.TRANSACTIONAL_ZOOKEEPER_PORT, Config.STORM_ZOOKEEPER_PORT);
-            CuratorFramework initter = Utils.newCuratorStarted(conf, servers, port);
+            ZookeeperAuthInfo auth = new ZookeeperAuthInfo(conf);
+            CuratorFramework initter = Utils.newCuratorStarted(conf, servers, port, auth);
+            _zkAcls = Utils.getWorkerACL(conf);
             try {
-                initter.create().creatingParentsIfNeeded().forPath(rootDir);
-            } catch(KeeperException.NodeExistsException e)  {
-                
+                TransactionalState.createNode(initter, transactionalRoot, null, null, null);
+            } catch (KeeperException.NodeExistsException e) {
+            }
+            try {
+                TransactionalState.createNode(initter, rootDir, null, _zkAcls, null);
+            } catch (KeeperException.NodeExistsException e) {
             }
-            
             initter.close();
                                     
-            _curator = Utils.newCuratorStarted(conf, servers, port, rootDir);
+            _curator = Utils.newCuratorStarted(conf, servers, port, rootDir, auth);
             _ser = new KryoValuesSerializer(conf);
             _des = new KryoValuesDeserializer(conf);
         } catch (Exception e) {
            throw new RuntimeException(e);
         }
     }
+
+    protected static String forPath(PathAndBytesable<String> builder, 
+            String path, byte[] data) throws Exception {
+        return (data == null) 
+            ? builder.forPath(path) 
+            : builder.forPath(path, data);
+    }
+
+    protected static void createNode(CuratorFramework curator, String path,
+            byte[] data, List<ACL> acls, CreateMode mode) throws Exception {
+        ProtectACLCreateModePathAndBytesable<String> builder =
+            curator.create().creatingParentsIfNeeded();
     
+        if (acls == null) {
+            if (mode == null ) {
+                TransactionalState.forPath(builder, path, data);
+            } else {
+                TransactionalState.forPath(builder.withMode(mode), path, data);
+            }
+            return;
+        }
+
+        TransactionalState.forPath(builder.withACL(acls), path, data);
+    }
+
     public void setData(String path, Object obj) {
         path = "/" + path;
         byte[] ser = _ser.serializeObject(obj);
@@ -78,10 +114,8 @@ public class TransactionalState {
             if(_curator.checkExists().forPath(path)!=null) {
                 _curator.setData().forPath(path, ser);
             } else {
-                _curator.create()
-                        .creatingParentsIfNeeded()
-                        .withMode(CreateMode.PERSISTENT)
-                        .forPath(path, ser);
+                TransactionalState.createNode(_curator, path, ser, _zkAcls,
+                        CreateMode.PERSISTENT);
             }
         } catch(Exception e) {
             throw new RuntimeException(e);

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/ui/InvalidRequestException.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/ui/InvalidRequestException.java b/storm-core/src/jvm/backtype/storm/ui/InvalidRequestException.java
new file mode 100644
index 0000000..9d0ee92
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/ui/InvalidRequestException.java
@@ -0,0 +1,20 @@
+package backtype.storm.ui;
+
+public class InvalidRequestException extends Exception {
+
+    public InvalidRequestException() {
+        super();
+    }
+
+    public InvalidRequestException(String msg) {
+        super(msg);
+    }
+
+    public InvalidRequestException(String msg, Throwable cause) {
+        super(msg, cause);
+    }
+
+    public InvalidRequestException(Throwable cause) {
+        super(cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/utils/DRPCClient.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/DRPCClient.java b/storm-core/src/jvm/backtype/storm/utils/DRPCClient.java
index cf38fb8..3218e49 100644
--- a/storm-core/src/jvm/backtype/storm/utils/DRPCClient.java
+++ b/storm-core/src/jvm/backtype/storm/utils/DRPCClient.java
@@ -17,68 +17,49 @@
  */
 package backtype.storm.utils;
 
+import backtype.storm.Config;
 import backtype.storm.generated.DRPCExecutionException;
 import backtype.storm.generated.DistributedRPC;
+import backtype.storm.generated.AuthorizationException;
 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 backtype.storm.security.auth.ThriftClient;
+import backtype.storm.security.auth.ThriftConnectionType;
+import org.apache.thrift.transport.TTransportException;
 
-public class DRPCClient implements DistributedRPC.Iface {
+import java.util.Map;
+
+public class DRPCClient extends ThriftClient implements DistributedRPC.Iface {
     private TTransport conn;
     private DistributedRPC.Client client;
     private String host;
     private int port;
     private Integer timeout;
 
-    public DRPCClient(String host, int port, Integer timeout) {
-        try {
-            this.host = host;
-            this.port = port;
-            this.timeout = timeout;
-            connect();
-        } catch(TException e) {
-            throw new RuntimeException(e);
-        }
-    }
-    
-    public DRPCClient(String host, int port) {
-        this(host, port, null);
+    public DRPCClient(Map conf, String host, int port) throws TTransportException {
+        this(conf, host, port, null);
     }
-    
-    private void connect() throws TException {
-        TSocket socket = new TSocket(host, port);
-        if(timeout!=null) {
-            socket.setTimeout(timeout);
-        }
-        conn = new TFramedTransport(socket);
-        client = new DistributedRPC.Client(new TBinaryProtocol(conn));
-        conn.open();
+
+    public DRPCClient(Map conf, String host, int port, Integer timeout) throws TTransportException {
+        super(conf, ThriftConnectionType.DRPC, host, port, timeout);
+        this.host = host;
+        this.port = port;
+        this.client = new DistributedRPC.Client(_protocol);
     }
-    
+        
     public String getHost() {
         return host;
     }
     
     public int getPort() {
         return port;
-    }   
+    }
     
-    public String execute(String func, String args) throws TException, DRPCExecutionException {
-        try {
-            if(client==null) connect();
-            return client.execute(func, args);
-        } catch(TException e) {
-            client = null;
-            throw e;
-        } catch(DRPCExecutionException e) {
-            client = null;
-            throw e;
-        }
+    public String execute(String func, String args) throws TException, DRPCExecutionException, AuthorizationException {
+        return client.execute(func, args);
     }
 
-    public void close() {
-        conn.close();
+    public DistributedRPC.Client getClient() {
+        return client;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/utils/LocalState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/LocalState.java b/storm-core/src/jvm/backtype/storm/utils/LocalState.java
index 0d0ae07..f412ff3 100644
--- a/storm-core/src/jvm/backtype/storm/utils/LocalState.java
+++ b/storm-core/src/jvm/backtype/storm/utils/LocalState.java
@@ -23,16 +23,19 @@ import java.io.File;
 import java.util.Map;
 import java.util.HashMap;
 import java.io.IOException;
-
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * A simple, durable, atomic K/V database. *Very inefficient*, should only be used for occasional reads/writes.
  * Every read/write hits disk.
  */
 public class LocalState {
+    public static Logger LOG = LoggerFactory.getLogger(LocalState.class);
     private VersionedStore _vs;
     
     public LocalState(String backingDir) throws IOException {
+        LOG.debug("New Local State for {}", backingDir);
         _vs = new VersionedStore(backingDir);
     }
     
@@ -83,8 +86,14 @@ public class LocalState {
     private void persist(Map<Object, Object> val, boolean cleanup) throws IOException {
         byte[] toWrite = Utils.serialize(val);
         String newPath = _vs.createVersion();
-        FileUtils.writeByteArrayToFile(new File(newPath), toWrite);
+        File file = new File(newPath);
+        FileUtils.writeByteArrayToFile(file, toWrite);
+        if (toWrite.length != file.length()) {
+            throw new IOException("Tried to serialize " + toWrite.length + 
+                    " bytes to " + file.getCanonicalPath() + ", but " +
+                    file.length() + " bytes were written.");
+        }
         _vs.succeedVersion(newPath);
         if(cleanup) _vs.cleanup(4);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java b/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
index e93acc8..273e232 100644
--- a/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
+++ b/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
@@ -19,6 +19,7 @@ package backtype.storm.utils;
 
 import backtype.storm.Config;
 import backtype.storm.security.auth.ThriftClient;
+import backtype.storm.security.auth.ThriftConnectionType;
 import backtype.storm.generated.Nimbus;
 import java.util.Map;
 import org.apache.thrift.transport.TTransportException;
@@ -32,8 +33,7 @@ public class NimbusClient extends ThriftClient {
     public static NimbusClient getConfiguredClient(Map conf) {
         try {
             String nimbusHost = (String) conf.get(Config.NIMBUS_HOST);
-            int nimbusPort = Utils.getInt(conf.get(Config.NIMBUS_THRIFT_PORT));
-            return new NimbusClient(conf, nimbusHost, nimbusPort);
+            return new NimbusClient(conf, nimbusHost);
         } catch (TTransportException ex) {
             throw new RuntimeException(ex);
         }
@@ -44,7 +44,12 @@ public class NimbusClient extends ThriftClient {
     }
 
     public NimbusClient(Map conf, String host, int port, Integer timeout) throws TTransportException {
-        super(conf, host, port, timeout);
+        super(conf, ThriftConnectionType.NIMBUS, host, port, timeout);
+        _client = new Nimbus.Client(_protocol);
+    }
+
+    public NimbusClient(Map conf, String host) throws TTransportException {
+        super(conf, ThriftConnectionType.NIMBUS, host, null, null);
         _client = new Nimbus.Client(_protocol);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/utils/TestUtils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/TestUtils.java b/storm-core/src/jvm/backtype/storm/utils/TestUtils.java
new file mode 100644
index 0000000..276559c
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/utils/TestUtils.java
@@ -0,0 +1,34 @@
+/**
+ * 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.utils;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.retry.ExponentialBackoffRetry;
+import java.util.Map;
+
+public class TestUtils extends Utils {
+
+    public static void testSetupBuilder(CuratorFrameworkFactory.Builder
+            builder, String zkStr, Map conf, ZookeeperAuthInfo auth)
+    {
+        setupBuilder(builder, zkStr, conf, auth);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/utils/Utils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/Utils.java b/storm-core/src/jvm/backtype/storm/utils/Utils.java
index a1fed96..c28d93a 100644
--- a/storm-core/src/jvm/backtype/storm/utils/Utils.java
+++ b/storm-core/src/jvm/backtype/storm/utils/Utils.java
@@ -17,6 +17,7 @@
  */
 package backtype.storm.utils;
 
+import java.io.BufferedReader;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.FileOutputStream;
@@ -45,7 +46,12 @@ import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.retry.ExponentialBackoffRetry;
 import org.apache.commons.lang.StringUtils;
 import org.apache.thrift.TException;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
 import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.yaml.snakeyaml.Yaml;
 import org.yaml.snakeyaml.constructor.SafeConstructor;
 
@@ -53,10 +59,13 @@ import backtype.storm.Config;
 import backtype.storm.generated.ComponentCommon;
 import backtype.storm.generated.ComponentObject;
 import backtype.storm.generated.StormTopology;
+import backtype.storm.generated.AuthorizationException;
+
 import clojure.lang.IFn;
 import clojure.lang.RT;
 
 public class Utils {
+    public static Logger LOG = LoggerFactory.getLogger(Utils.class);    
     public static final String DEFAULT_STREAM_ID = "default";
 
     public static Object newInstance(String klass) {
@@ -254,7 +263,7 @@ public class Utils {
         return ret;
     }
 
-    public static void downloadFromMaster(Map conf, String file, String localFile) throws IOException, TException {
+    public static void downloadFromMaster(Map conf, String file, String localFile) throws AuthorizationException, IOException, TException {
         NimbusClient client = NimbusClient.getConfiguredClient(conf);
         String id = client.getClient().beginFileDownload(file);
         WritableByteChannel out = Channels.newChannel(new FileOutputStream(localFile));
@@ -307,6 +316,8 @@ public class Utils {
             return (Integer) o;
         } else if (o instanceof Short) {
             return ((Short) o).intValue();
+        } else if (o instanceof String) {
+            return Integer.parseInt((String) o);
         } else {
             throw new IllegalArgumentException("Don't know how to convert " + o + " + to int");
         }
@@ -316,11 +327,6 @@ public class Utils {
         return UUID.randomUUID().getLeastSignificantBits();
     }
     
-    
-    public static CuratorFramework newCurator(Map conf, List<String> servers, Object port, String root) {
-        return newCurator(conf, servers, port, root, null);
-    }
-
     public static class BoundedExponentialBackoffRetry extends ExponentialBackoffRetry {
 
         protected final int maxRetryInterval;
@@ -350,32 +356,39 @@ public class Utils {
             serverPorts.add(zkServer + ":" + Utils.getInt(port));
         }
         String zkStr = StringUtils.join(serverPorts, ",") + root;
-        CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder()
-                .connectString(zkStr)
-                .connectionTimeoutMs(Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT)))
-                .sessionTimeoutMs(Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT)))
-                .retryPolicy(new BoundedExponentialBackoffRetry(
+        CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder();
+
+        setupBuilder(builder, zkStr, conf, auth);
+        
+        return builder.build();
+    }
+
+    protected static void setupBuilder(CuratorFrameworkFactory.Builder builder, String zkStr, Map conf, ZookeeperAuthInfo auth)
+    {
+        builder.connectString(zkStr)
+            .connectionTimeoutMs(Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT)))
+            .sessionTimeoutMs(Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT)))
+            .retryPolicy(new BoundedExponentialBackoffRetry(
                             Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL)),
                             Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_TIMES)),
                             Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING))));
-        if(auth!=null && auth.scheme!=null) {
+        if(auth!=null && auth.scheme!=null && auth.payload!=null) {
             builder = builder.authorization(auth.scheme, auth.payload);
         }
-        return builder.build();
     }
 
-    public static CuratorFramework newCurator(Map conf, List<String> servers, Object port) {
-        return newCurator(conf, servers, port, "");
+    public static CuratorFramework newCurator(Map conf, List<String> servers, Object port, ZookeeperAuthInfo auth) {
+        return newCurator(conf, servers, port, "", auth);
     }
 
-    public static CuratorFramework newCuratorStarted(Map conf, List<String> servers, Object port, String root) {
-        CuratorFramework ret = newCurator(conf, servers, port, root);
+    public static CuratorFramework newCuratorStarted(Map conf, List<String> servers, Object port, String root, ZookeeperAuthInfo auth) {
+        CuratorFramework ret = newCurator(conf, servers, port, root, auth);
         ret.start();
         return ret;
     }
 
-    public static CuratorFramework newCuratorStarted(Map conf, List<String> servers, Object port) {
-        CuratorFramework ret = newCurator(conf, servers, port);
+    public static CuratorFramework newCuratorStarted(Map conf, List<String> servers, Object port, ZookeeperAuthInfo auth) {
+        CuratorFramework ret = newCurator(conf, servers, port, auth);
         ret.start();
         return ret;
     }    
@@ -413,6 +426,18 @@ public class Utils {
         return ret;
     }
 
+    public static void readAndLogStream(String prefix, InputStream in) {
+        try {
+            BufferedReader r = new BufferedReader(new InputStreamReader(in));
+            String line = null;
+            while ((line = r.readLine())!= null) {
+                LOG.info("{}:{}", prefix, line);
+            }
+        } catch (IOException e) {
+            LOG.warn("Error whiel trying to log stream", e);
+        }
+    }
+
     public static boolean exceptionCauseIsInstanceOf(Class klass, Throwable throwable) {
         Throwable t = throwable;
         while(t != null) {
@@ -423,4 +448,67 @@ public class Utils {
         }
         return false;
     }
+
+    /**
+     * Is the cluster configured to interact with ZooKeeper in a secure way?
+     * This only works when called from within Nimbus or a Supervisor process.
+     * @param conf the storm configuration, not the topology configuration
+     * @return true if it is configured else false.
+     */
+    public static boolean isZkAuthenticationConfiguredStormServer(Map conf) {
+        return null != System.getProperty("java.security.auth.login.config")
+            || (conf != null
+                && conf.get(Config.STORM_ZOOKEEPER_AUTH_SCHEME) != null
+                && ! ((String)conf.get(Config.STORM_ZOOKEEPER_AUTH_SCHEME)).isEmpty());
+    }
+
+    /**
+     * Is the topology configured to have ZooKeeper authentication.
+     * @param conf the topology configuration
+     * @return true if ZK is configured else false
+     */
+    public static boolean isZkAuthenticationConfiguredTopology(Map conf) {
+        return (conf != null
+                && conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME) != null
+                && ! ((String)conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME)).isEmpty());
+    }
+
+    public static List<ACL> getWorkerACL(Map conf) {
+        //This is a work around to an issue with ZK where a sasl super user is not super unless there is an open SASL ACL so we are trying to give the correct perms
+        if (!isZkAuthenticationConfiguredTopology(conf)) {
+            return null;
+        }
+        String stormZKUser = (String)conf.get(Config.STORM_ZOOKEEPER_SUPERACL);
+        if (stormZKUser == null) {
+           throw new IllegalArgumentException("Authentication is enabled but "+Config.STORM_ZOOKEEPER_SUPERACL+" is not set");
+        }
+        String[] split = stormZKUser.split(":",2);
+        if (split.length != 2) {
+          throw new IllegalArgumentException(Config.STORM_ZOOKEEPER_SUPERACL+" does not appear to be in the form scheme:acl, i.e. sasl:storm-user");
+        }
+        ArrayList<ACL> ret = new ArrayList<ACL>(ZooDefs.Ids.CREATOR_ALL_ACL);
+        ret.add(new ACL(ZooDefs.Perms.ALL, new Id(split[0], split[1])));
+        return ret;
+    }
+
+   public static String threadDump() {
+       final StringBuilder dump = new StringBuilder();
+       final java.lang.management.ThreadMXBean threadMXBean =  java.lang.management.ManagementFactory.getThreadMXBean();
+       final java.lang.management.ThreadInfo[] threadInfos = threadMXBean.getThreadInfo(threadMXBean.getAllThreadIds(), 100);
+       for (java.lang.management.ThreadInfo threadInfo : threadInfos) {
+           dump.append('"');
+           dump.append(threadInfo.getThreadName());
+           dump.append("\" ");
+           final Thread.State state = threadInfo.getThreadState();
+           dump.append("\n   java.lang.Thread.State: ");
+           dump.append(state);
+           final StackTraceElement[] stackTraceElements = threadInfo.getStackTrace();
+           for (final StackTraceElement stackTraceElement : stackTraceElements) {
+               dump.append("\n        at ");
+               dump.append(stackTraceElement);
+           }
+           dump.append("\n\n");
+       }
+       return dump.toString();
+   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/utils/ZookeeperAuthInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/ZookeeperAuthInfo.java b/storm-core/src/jvm/backtype/storm/utils/ZookeeperAuthInfo.java
index a5a2e9a..d972135 100644
--- a/storm-core/src/jvm/backtype/storm/utils/ZookeeperAuthInfo.java
+++ b/storm-core/src/jvm/backtype/storm/utils/ZookeeperAuthInfo.java
@@ -27,8 +27,13 @@ public class ZookeeperAuthInfo {
     public byte[] payload = null;
     
     public ZookeeperAuthInfo(Map conf) {
-        String scheme = (String) conf.get(Config.STORM_ZOOKEEPER_AUTH_SCHEME);
-        String payload = (String) conf.get(Config.STORM_ZOOKEEPER_AUTH_PAYLOAD);
+        String scheme = (String) conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME);
+        String payload = (String) conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD);
+
+        if (scheme == null || payload == null) {
+            scheme = (String) conf.get(Config.STORM_ZOOKEEPER_AUTH_SCHEME);
+            payload = (String) conf.get(Config.STORM_ZOOKEEPER_AUTH_PAYLOAD);
+        }
         if(scheme!=null) {
             this.scheme = scheme;
             if(payload != null) {

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/backtype/storm/utils/ZookeeperServerCnxnFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/ZookeeperServerCnxnFactory.java b/storm-core/src/jvm/backtype/storm/utils/ZookeeperServerCnxnFactory.java
new file mode 100644
index 0000000..08a763a
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/utils/ZookeeperServerCnxnFactory.java
@@ -0,0 +1,84 @@
+/**
+ * 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.utils;
+
+import java.io.IOException;
+import java.net.BindException;
+import java.net.InetSocketAddress;
+
+import org.apache.zookeeper.server.NIOServerCnxnFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ZookeeperServerCnxnFactory {
+	private static final Logger LOG = LoggerFactory.getLogger(ZookeeperServerCnxnFactory.class);
+	int _port;
+	NIOServerCnxnFactory _factory;
+	
+	public ZookeeperServerCnxnFactory(int port, int maxClientCnxns)  {
+		//port range
+		int max;
+		if (port <= 0) {
+			_port = 2000;
+			max = 65535;
+		} else {
+			_port = port;
+			max = port;
+		}
+
+		try {
+			_factory = new NIOServerCnxnFactory();
+		} catch (IOException e) {
+			_port = 0;
+			_factory = null;
+			e.printStackTrace();
+			throw new RuntimeException(e.getMessage());
+		}
+		
+		//look for available port 
+		for (; _port <= max; _port++) {
+			try {
+				_factory.configure(new InetSocketAddress(_port), maxClientCnxns);
+				LOG.debug("Zookeeper server successfully binded at port "+_port);
+				break;
+			} catch (BindException e1) {
+			} catch (IOException e2) {
+				_port = 0;
+				_factory = null;
+				e2.printStackTrace();
+				throw new RuntimeException(e2.getMessage());
+			} 
+		} 		
+
+		if (_port > max) {
+			_port = 0;
+			_factory = null;
+			LOG.error("Failed to find a port for Zookeeper");
+			throw new RuntimeException("No port is available to launch an inprocess zookeeper.");
+		}
+	}
+	
+	public int port() {
+		return _port;
+	}
+		
+	public NIOServerCnxnFactory factory() {
+		return _factory;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/storm/trident/drpc/ReturnResultsReducer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/storm/trident/drpc/ReturnResultsReducer.java b/storm-core/src/jvm/storm/trident/drpc/ReturnResultsReducer.java
index 4ebb667..d49371a 100644
--- a/storm-core/src/jvm/storm/trident/drpc/ReturnResultsReducer.java
+++ b/storm-core/src/jvm/storm/trident/drpc/ReturnResultsReducer.java
@@ -20,6 +20,7 @@ package storm.trident.drpc;
 import backtype.storm.Config;
 import backtype.storm.drpc.DRPCInvocationsClient;
 import backtype.storm.generated.DistributedRPCInvocations;
+import backtype.storm.generated.AuthorizationException;
 import backtype.storm.utils.ServiceRegistry;
 import backtype.storm.utils.Utils;
 import java.util.ArrayList;
@@ -28,6 +29,7 @@ import java.util.List;
 import java.util.Map;
 import org.apache.commons.lang.builder.ToStringBuilder;
 import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransportException;
 import org.json.simple.JSONValue;
 import storm.trident.drpc.ReturnResultsReducer.ReturnResultsState;
 import storm.trident.operation.MultiReducer;
@@ -47,12 +49,13 @@ public class ReturnResultsReducer implements MultiReducer<ReturnResultsState> {
         }
     }
     boolean local;
-
+    Map conf;
     Map<List, DRPCInvocationsClient> _clients = new HashMap<List, DRPCInvocationsClient>();
     
     
     @Override
     public void prepare(Map conf, TridentMultiReducerContext context) {
+        this.conf = conf;
         local = conf.get(Config.STORM_CLUSTER_MODE).equals("local");
     }
 
@@ -89,7 +92,11 @@ public class ReturnResultsReducer implements MultiReducer<ReturnResultsState> {
                 }};
 
                 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);
             }
@@ -98,6 +105,8 @@ public class ReturnResultsReducer implements MultiReducer<ReturnResultsState> {
                 client.result(id, result);
             } catch(TException e) {
                 collector.reportError(e);
+            } catch (AuthorizationException aze) {
+                collector.reportError(aze);                
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/storm/trident/topology/state/TestTransactionalState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/storm/trident/topology/state/TestTransactionalState.java b/storm-core/src/jvm/storm/trident/topology/state/TestTransactionalState.java
new file mode 100644
index 0000000..ff3edb6
--- /dev/null
+++ b/storm-core/src/jvm/storm/trident/topology/state/TestTransactionalState.java
@@ -0,0 +1,47 @@
+/**
+ * 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 storm.trident.topology.state;
+
+import java.util.List;
+import java.util.Map;
+
+import backtype.storm.utils.ZookeeperAuthInfo;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.data.ACL;
+
+/**
+ * Facilitates testing of non-public methods in the parent class.
+ */
+public class TestTransactionalState extends TransactionalState {
+
+    /**
+     * Matching constructor in absence of a default constructor in the parent
+     * class.
+     */
+    protected TestTransactionalState(Map conf, String id, String subroot) {
+        super(conf, id, subroot);
+    }
+
+    public static void createNode(CuratorFramework curator, 
+            String rootDir, byte[] data, List<ACL> acls, CreateMode mode)
+            throws Exception {
+       TransactionalState.createNode(curator, rootDir, data, acls, mode);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/jvm/storm/trident/topology/state/TransactionalState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/storm/trident/topology/state/TransactionalState.java b/storm-core/src/jvm/storm/trident/topology/state/TransactionalState.java
index 5325137..5fba1a2 100644
--- a/storm-core/src/jvm/storm/trident/topology/state/TransactionalState.java
+++ b/storm-core/src/jvm/storm/trident/topology/state/TransactionalState.java
@@ -20,7 +20,12 @@ package storm.trident.topology.state;
 
 import backtype.storm.Config;
 import backtype.storm.utils.Utils;
+import backtype.storm.utils.ZookeeperAuthInfo;
+
 import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.api.ProtectACLCreateModePathAndBytesable;
+import org.apache.curator.framework.api.PathAndBytesable;
+
 import java.io.UnsupportedEncodingException;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -28,10 +33,14 @@ import java.util.List;
 import java.util.Map;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
 import org.json.simple.JSONValue;
 
 public class TransactionalState {
     CuratorFramework _curator;
+    List<ACL> _zkAcls = null;
     
     public static TransactionalState newUserState(Map conf, String id) {
         return new TransactionalState(conf, id, "user");
@@ -44,24 +53,53 @@ public class TransactionalState {
     protected TransactionalState(Map conf, String id, String subroot) {
         try {
             conf = new HashMap(conf);
-            String rootDir = conf.get(Config.TRANSACTIONAL_ZOOKEEPER_ROOT) + "/" + id + "/" + subroot;
+            String transactionalRoot = (String)conf.get(Config.TRANSACTIONAL_ZOOKEEPER_ROOT);
+            String rootDir = transactionalRoot + "/" + id + "/" + subroot;
             List<String> servers = (List<String>) getWithBackup(conf, Config.TRANSACTIONAL_ZOOKEEPER_SERVERS, Config.STORM_ZOOKEEPER_SERVERS);
             Object port = getWithBackup(conf, Config.TRANSACTIONAL_ZOOKEEPER_PORT, Config.STORM_ZOOKEEPER_PORT);
-            CuratorFramework initter = Utils.newCuratorStarted(conf, servers, port);
+            ZookeeperAuthInfo auth = new ZookeeperAuthInfo(conf);
+            CuratorFramework initter = Utils.newCuratorStarted(conf, servers, port, auth);
+            _zkAcls = Utils.getWorkerACL(conf);
+            try {
+                TransactionalState.createNode(initter, transactionalRoot, null, null, null);
+            } catch (KeeperException.NodeExistsException e) {
+            }
             try {
-                initter.create().creatingParentsIfNeeded().forPath(rootDir);
-            } catch(KeeperException.NodeExistsException e)  {
-                
+                TransactionalState.createNode(initter, rootDir, null, _zkAcls, null);
+            } catch (KeeperException.NodeExistsException e) {
             }
-            
             initter.close();
                                     
-            _curator = Utils.newCuratorStarted(conf, servers, port, rootDir);
+            _curator = Utils.newCuratorStarted(conf, servers, port, rootDir, auth);
         } catch (Exception e) {
            throw new RuntimeException(e);
         }
     }
+
+    protected static String forPath(PathAndBytesable<String> builder, 
+            String path, byte[] data) throws Exception {
+        return (data == null) 
+            ? builder.forPath(path) 
+            : builder.forPath(path, data);
+    }
+
+    protected static void createNode(CuratorFramework curator, String path,
+            byte[] data, List<ACL> acls, CreateMode mode) throws Exception {
+        ProtectACLCreateModePathAndBytesable<String> builder =
+            curator.create().creatingParentsIfNeeded();
     
+        if (acls == null) {
+            if (mode == null ) {
+                TransactionalState.forPath(builder, path, data);
+            } else {
+                TransactionalState.forPath(builder.withMode(mode), path, data);
+            }
+            return;
+        }
+
+        TransactionalState.forPath(builder.withACL(acls), path, data);
+    }
+
     public void setData(String path, Object obj) {
         path = "/" + path;
         byte[] ser;
@@ -74,10 +112,8 @@ public class TransactionalState {
             if(_curator.checkExists().forPath(path)!=null) {
                 _curator.setData().forPath(path, ser);
             } else {
-                _curator.create()
-                        .creatingParentsIfNeeded()
-                        .withMode(CreateMode.PERSISTENT)
-                        .forPath(path, ser);
+                TransactionalState.createNode(_curator, path, ser, _zkAcls,
+                        CreateMode.PERSISTENT);
             }
         } catch(Exception e) {
             throw new RuntimeException(e);

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/native/worker-launcher/.autom4te.cfg
----------------------------------------------------------------------
diff --git a/storm-core/src/native/worker-launcher/.autom4te.cfg b/storm-core/src/native/worker-launcher/.autom4te.cfg
new file mode 100644
index 0000000..1ec584f
--- /dev/null
+++ b/storm-core/src/native/worker-launcher/.autom4te.cfg
@@ -0,0 +1,42 @@
+#
+# 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.
+#
+
+#
+# autom4te configuration for worker-launcher
+#
+
+begin-language: "Autoheader-preselections"
+args: --no-cache 
+end-language: "Autoheader-preselections"
+
+begin-language: "Automake-preselections"
+args: --no-cache 
+end-language: "Automake-preselections"
+
+begin-language: "Autoreconf-preselections"
+args: --no-cache 
+end-language: "Autoreconf-preselections"
+
+begin-language: "Autoconf-without-aclocal-m4"
+args: --no-cache 
+end-language: "Autoconf-without-aclocal-m4"
+
+begin-language: "Autoconf"
+args: --no-cache 
+end-language: "Autoconf"
+

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/native/worker-launcher/.deps/worker-launcher.Po
----------------------------------------------------------------------
diff --git a/storm-core/src/native/worker-launcher/.deps/worker-launcher.Po b/storm-core/src/native/worker-launcher/.deps/worker-launcher.Po
new file mode 100644
index 0000000..9ce06a8
--- /dev/null
+++ b/storm-core/src/native/worker-launcher/.deps/worker-launcher.Po
@@ -0,0 +1 @@
+# dummy

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/native/worker-launcher/Makefile.am
----------------------------------------------------------------------
diff --git a/storm-core/src/native/worker-launcher/Makefile.am b/storm-core/src/native/worker-launcher/Makefile.am
new file mode 100644
index 0000000..c9183c0
--- /dev/null
+++ b/storm-core/src/native/worker-launcher/Makefile.am
@@ -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.
+
+AM_CFLAGS=-I$(srcdir)/impl -Wall -g -Werror
+
+# Define the programs that need to be built
+bin_PROGRAMS = worker-launcher
+check_PROGRAMS = test-worker-launcher
+
+TESTS = test-worker-launcher
+
+# Define the sources for the common files
+common_SOURCES = impl/configuration.c impl/worker-launcher.c
+
+# Define the sources for the real executable
+worker_launcher_SOURCES = $(common_SOURCES) impl/main.c
+
+# Define the sources for the test executable
+test_worker_launcher_SOURCES = $(common_SOURCES) test/test-worker-launcher.c

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/native/worker-launcher/configure.ac
----------------------------------------------------------------------
diff --git a/storm-core/src/native/worker-launcher/configure.ac b/storm-core/src/native/worker-launcher/configure.ac
new file mode 100644
index 0000000..ab1ef49
--- /dev/null
+++ b/storm-core/src/native/worker-launcher/configure.ac
@@ -0,0 +1,50 @@
+# 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.
+#
+#                                               -*- Autoconf -*-
+# Process this file with autoconf to produce a configure script.
+
+AC_PREREQ(2.59)
+AC_INIT(worker-launcher, 1.0.0, storm-user@googlegroups.com)
+AC_GNU_SOURCE
+
+AM_INIT_AUTOMAKE([subdir-objects foreign no-dist])
+
+AC_CONFIG_SRCDIR([impl/worker-launcher.c])
+AC_CONFIG_FILES([Makefile])
+
+AC_PREFIX_DEFAULT(`pwd`/../install)
+
+# Checks for programs.
+AC_PROG_CC
+AM_PROG_CC_C_O
+
+# Checks for libraries.
+
+# Checks for header files.
+AC_LANG(C)
+AC_CHECK_HEADERS([unistd.h])
+
+# Checks for typedefs, structures, and compiler characteristics.
+AC_HEADER_STDBOOL
+AC_C_CONST
+AC_TYPE_OFF_T
+AC_TYPE_SIZE_T
+AC_FUNC_STRERROR_R
+
+# Checks for library functions.
+AC_CHECK_FUNCS([mkdir uname])
+AC_OUTPUT

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/native/worker-launcher/impl/configuration.c
----------------------------------------------------------------------
diff --git a/storm-core/src/native/worker-launcher/impl/configuration.c b/storm-core/src/native/worker-launcher/impl/configuration.c
new file mode 100644
index 0000000..7b7a3c1
--- /dev/null
+++ b/storm-core/src/native/worker-launcher/impl/configuration.c
@@ -0,0 +1,340 @@
+/**
+ * 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.
+ */
+
+// ensure we get the posix version of dirname by including this first
+#include <libgen.h> 
+
+#include "configuration.h"
+#include "worker-launcher.h"
+
+#include <errno.h>
+#include <limits.h>
+#include <unistd.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <sys/stat.h>
+#include <sys/types.h>
+
+#define MAX_SIZE 10
+
+struct confentry {
+  const char *key;
+  const char *value;
+};
+
+struct configuration {
+  int size;
+  struct confentry **confdetails;
+};
+
+struct configuration config={.size=0, .confdetails=NULL};
+
+//clean up method for freeing configuration
+void free_configurations() {
+  int i = 0;
+  for (i = 0; i < config.size; i++) {
+    if (config.confdetails[i]->key != NULL) {
+      free((void *)config.confdetails[i]->key);
+      config.confdetails[i]->key = NULL;
+    }
+    if (config.confdetails[i]->value != NULL) {
+      free((void *)config.confdetails[i]->value);
+      config.confdetails[i]->value = NULL;
+    }
+    free(config.confdetails[i]);
+    config.confdetails[i] = NULL;
+  }
+  if (config.size > 0) {
+    free(config.confdetails);
+    config.confdetails = NULL;
+  }
+  config.size = 0;
+}
+
+/**
+ * Is the file/directory only writable by root.
+ * Returns 1 if true
+ */
+static int is_only_root_writable(const char *file) {
+  struct stat file_stat;
+  if (stat(file, &file_stat) != 0) {
+    fprintf(ERRORFILE, "Can't stat file %s - %s\n", file, strerror(errno));
+    return 0;
+  }
+  if (file_stat.st_uid != 0) {
+    fprintf(ERRORFILE, "File %s must be owned by root, but is owned by %d\n",
+            file, file_stat.st_uid);
+    return 0;
+  }
+  if ((file_stat.st_mode & (S_IWGRP | S_IWOTH)) != 0) {
+    fprintf(ERRORFILE, 
+	    "File %s must not be world or group writable, but is %03o\n",
+	    file, file_stat.st_mode & (~S_IFMT));
+    return 0;
+  }
+  return 1;
+}
+
+/**
+ * Ensure that the configuration file and all of the containing directories
+ * are only writable by root. Otherwise, an attacker can change the 
+ * configuration and potentially cause damage.
+ * returns 0 if permissions are ok
+ */
+int check_configuration_permissions(const char* file_name) {
+  // copy the input so that we can modify it with dirname
+  char* dir = strndup(file_name, PATH_MAX);
+  char* buffer = dir;
+  do {
+    if (!is_only_root_writable(dir)) {
+      free(buffer);
+      buffer = NULL;
+      return -1;
+    }
+    dir = dirname(dir);
+  } while (strcmp(dir, "/") != 0);
+  free(buffer);
+  buffer = NULL;
+  return 0;
+}
+
+//function used to load the configurations present in the secure config
+void read_config(const char* file_name) {
+  FILE *conf_file;
+  char *line;
+  char *equaltok;
+  char *temp_equaltok;
+  size_t linesize = 1000;
+  int size_read = 0;
+
+  if (file_name == NULL) {
+    fprintf(ERRORFILE, "Null configuration filename passed in\n");
+    exit(INVALID_CONFIG_FILE);
+  }
+
+  #ifdef DEBUG
+    fprintf(LOGFILE, "read_config :Conf file name is : %s \n", file_name);
+  #endif
+
+  //allocate space for ten configuration items.
+  config.confdetails = (struct confentry **) malloc(sizeof(struct confentry *)
+      * MAX_SIZE);
+  if (config.confdetails == NULL) {
+      fprintf(ERRORFILE, "malloc failed while reading configuration file.\n");
+      exit(OUT_OF_MEMORY);
+  }
+  config.size = 0;
+  conf_file = fopen(file_name, "r");
+  if (conf_file == NULL) {
+    fprintf(ERRORFILE, "Invalid conf file provided : %s \n", file_name);
+    exit(INVALID_CONFIG_FILE);
+  }
+  while(!feof(conf_file)) {
+    line = (char *) malloc(linesize);
+    if(line == NULL) {
+      fprintf(ERRORFILE, "malloc failed while reading configuration file.\n");
+      exit(OUT_OF_MEMORY);
+    }
+    size_read = getline(&line,&linesize,conf_file);
+ 
+    //feof returns true only after we read past EOF.
+    //so a file with no new line, at last can reach this place
+    //if size_read returns negative check for eof condition
+    if (size_read == -1) {
+      free(line);
+      line = NULL;
+      if(!feof(conf_file)){
+        exit(INVALID_CONFIG_FILE);
+      } else {
+        break;
+      }
+    }
+    int eol = strlen(line) - 1;
+    if(line[eol] == '\n') {
+        //trim the ending new line
+        line[eol] = '\0';
+    }
+    //comment line
+    if(line[0] == '#') {
+      free(line);
+      line = NULL;
+      continue;
+    }
+    //tokenize first to get key and list of values.
+    //if no equals is found ignore this line, can be an empty line also
+    equaltok = strtok_r(line, "=", &temp_equaltok);
+    if(equaltok == NULL) {
+      free(line);
+      line = NULL;
+      continue;
+    }
+    config.confdetails[config.size] = (struct confentry *) malloc(
+            sizeof(struct confentry));
+    if(config.confdetails[config.size] == NULL) {
+      fprintf(LOGFILE,
+          "Failed allocating memory for single configuration item\n");
+      goto cleanup;
+    }
+
+    #ifdef DEBUG
+      fprintf(LOGFILE, "read_config : Adding conf key : %s \n", equaltok);
+    #endif
+
+    memset(config.confdetails[config.size], 0, sizeof(struct confentry));
+    const size_t key_tok_len = strlen(equaltok);
+    config.confdetails[config.size]->key = (char *) malloc(
+            sizeof(char) * (key_tok_len+1));
+    if (config.confdetails[config.size]->key == NULL) {
+      fprintf(LOGFILE,
+          "Failed allocating memory for single configuration item\n");
+      goto cleanup;
+    }
+    memset((void*)config.confdetails[config.size]->key, '\0', key_tok_len+1);
+    strncpy((char *)config.confdetails[config.size]->key, equaltok, key_tok_len);
+    equaltok = strtok_r(NULL, "=", &temp_equaltok);
+    if (equaltok == NULL) {
+      fprintf(LOGFILE, "configuration tokenization failed \n");
+      goto cleanup;
+    }
+    //means value is commented so don't store the key
+    if(equaltok[0] == '#') {
+      free(line);
+      line = NULL;
+      free((void *)config.confdetails[config.size]->key);
+      config.confdetails[config.size]->key = NULL;
+      free(config.confdetails[config.size]);
+      config.confdetails[config.size] = NULL;
+      continue;
+    }
+
+    #ifdef DEBUG
+      fprintf(LOGFILE, "read_config : Adding conf value : %s \n", equaltok);
+    #endif
+
+    const size_t val_tok_len = strlen(equaltok);
+    config.confdetails[config.size]->value = (char *) malloc(
+            sizeof(char) * (val_tok_len+1));
+    if (config.confdetails[config.size]->value == NULL) {
+      fprintf(LOGFILE,
+          "Failed allocating memory for single configuration item\n");
+      goto cleanup;
+    }
+    memset((void *)config.confdetails[config.size]->value, '\0', val_tok_len+1);
+    strncpy((char *)config.confdetails[config.size]->value, equaltok, val_tok_len);
+    if((config.size + 1) % MAX_SIZE  == 0) {
+      config.confdetails = (struct confentry **) realloc(config.confdetails,
+          sizeof(struct confentry **) * (MAX_SIZE + config.size));
+      if (config.confdetails == NULL) {
+        fprintf(LOGFILE,
+            "Failed re-allocating memory for configuration items\n");
+        goto cleanup;
+      }
+    }
+    if(config.confdetails[config.size] )
+    config.size++;
+    free(line);
+    line = NULL;
+  }
+ 
+  //close the file
+  fclose(conf_file);
+
+  if (config.size == 0) {
+    fprintf(ERRORFILE, "Invalid configuration provided in %s\n", file_name);
+    exit(INVALID_CONFIG_FILE);
+  }
+
+  //clean up allocated file name
+  return;
+  //free spaces alloced.
+  cleanup:
+  if (line != NULL) {
+    free(line);
+    line = NULL;
+  }
+  fclose(conf_file);
+  free_configurations();
+  return;
+}
+
+/*
+ * function used to get a configuration value.
+ * The function for the first time populates the configuration details into
+ * array, next time onwards used the populated array.
+ *
+ */
+char * get_value(const char* key) {
+  int count;
+  for (count = 0; count < config.size; count++) {
+    if (strcmp(config.confdetails[count]->key, key) == 0) {
+      return strdup(config.confdetails[count]->value);
+    }
+  }
+  return NULL;
+}
+
+/**
+ * Function to return an array of values for a key.
+ * Value delimiter is assumed to be a comma.
+ */
+char ** get_values(const char * key) {
+  char *value = get_value(key);
+  return extract_values(value);
+}
+
+/**
+ * Extracts array of values from the comma separated list of values.
+ */
+char ** extract_values(char *value) {
+  char ** toPass = NULL;
+  char *tempTok = NULL;
+  char *tempstr = NULL;
+  int size = 0;
+  int toPassSize = MAX_SIZE;
+
+  //first allocate any array of 10
+  if(value != NULL) {
+    toPass = (char **) malloc(sizeof(char *) * toPassSize);
+    tempTok = strtok_r((char *)value, ",", &tempstr);
+    while (tempTok != NULL) {
+      toPass[size++] = tempTok;
+      if(size == toPassSize) {
+        toPassSize += MAX_SIZE;
+        toPass = (char **) realloc(toPass,(sizeof(char *) * toPassSize));
+      }
+      tempTok = strtok_r(NULL, ",", &tempstr);
+    }
+  }
+  if (size > 0) {
+    toPass[size] = NULL;
+  }
+  return toPass;
+}
+
+// free an entry set of values
+void free_values(char** values) {
+  if (*values != NULL) {
+    free(*values);
+    *values = NULL;
+  }
+  if (values != NULL) {
+    free(values);
+    values = NULL;
+  }
+}


[02/14] STORM-216: Added Authentication and Authorization.

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj b/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj
index c7d7411..6fdd485 100644
--- a/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj
+++ b/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj
@@ -19,36 +19,40 @@
   (:import [org.apache.thrift TException])
   (:import [org.apache.thrift.transport TTransportException])
   (:import [java.nio ByteBuffer])
+  (:import [java.security Principal AccessController])
+  (:import [javax.security.auth Subject])
+  (:import [java.net InetAddress])
   (:import [backtype.storm Config])
+  (:import [backtype.storm.generated AuthorizationException])
   (:import [backtype.storm.utils NimbusClient])
+  (:import [backtype.storm.security.auth.authorizer SimpleWhitelistAuthorizer SimpleACLAuthorizer])
   (:import [backtype.storm.security.auth AuthUtils ThriftServer ThriftClient 
-                                         ReqContext])
+            ReqContext SimpleTransportPlugin KerberosPrincipalToLocal ThriftConnectionType])
   (:use [backtype.storm bootstrap util])
   (:use [backtype.storm.daemon common])
   (:use [backtype.storm bootstrap testing])
-  (:import [backtype.storm.generated Nimbus Nimbus$Client])
-  )
+  (:import [backtype.storm.generated Nimbus Nimbus$Client]))
 
 (bootstrap)
 
-(def nimbus-timeout (Integer. 120))
+(defn mk-principal [name]
+  (reify Principal
+    (equals [this other]
+      (= name (.getName other)))
+    (getName [this] name)
+    (toString [this] name)
+    (hashCode [this] (.hashCode name))))
+
+(defn mk-subject [name]
+  (Subject. true #{(mk-principal name)} #{} #{}))
 
-(defn mk-authorization-handler [storm-conf]
-  (let [klassname (storm-conf NIMBUS-AUTHORIZER) 
-        aznClass (if klassname (Class/forName klassname))
-        aznHandler (if aznClass (.newInstance aznClass))] 
-    (if aznHandler (.prepare aznHandler storm-conf))
-    (log-debug "authorization class name:" klassname
-               " class:" aznClass
-               " handler:" aznHandler)
-    aznHandler
-    ))
+(def nimbus-timeout (Integer. 120))
 
 (defn nimbus-data [storm-conf inimbus]
   (let [forced-scheduler (.getForcedScheduler inimbus)]
     {:conf storm-conf
      :inimbus inimbus
-     :authorization-handler (mk-authorization-handler storm-conf)
+     :authorization-handler (mk-authorization-handler (storm-conf NIMBUS-AUTHORIZER) storm-conf)
      :submitted-count (atom 0)
      :storm-cluster-state nil
      :submit-lock (Object.)
@@ -61,71 +65,75 @@
      :scheduler nil
      }))
 
-(defn check-authorization! [nimbus storm-name storm-conf operation]
-  (let [aclHandler (:authorization-handler nimbus)]
-    (log-debug "check-authorization with handler: " aclHandler)
-    (if aclHandler
-        (if-not (.permit aclHandler 
-                  (ReqContext/context) 
-                  operation 
-                  (if storm-conf storm-conf {TOPOLOGY-NAME storm-name}))
-          (throw (RuntimeException. (str operation " on topology " storm-name " is not authorized")))
-          ))))
-
-(defn dummy-service-handler [conf inimbus]
-  (let [nimbus (nimbus-data conf inimbus)]
-    (reify Nimbus$Iface
-      (^void submitTopologyWithOpts [this ^String storm-name ^String uploadedJarLocation ^String serializedConf ^StormTopology topology
-                                     ^SubmitOptions submitOptions]
-        (check-authorization! nimbus storm-name nil "submitTopology"))
-      
-      (^void killTopology [this ^String storm-name]
-        (check-authorization! nimbus storm-name nil "killTopology"))
-      
-      (^void killTopologyWithOpts [this ^String storm-name ^KillOptions options]
-        (check-authorization! nimbus storm-name nil "killTopology"))
-
-      (^void rebalance [this ^String storm-name ^RebalanceOptions options]
-        (check-authorization! nimbus storm-name nil "rebalance"))
-
-      (activate [this storm-name]
-        (check-authorization! nimbus storm-name nil "activate"))
-
-      (deactivate [this storm-name]
-        (check-authorization! nimbus storm-name nil "deactivate"))
-
-      (beginFileUpload [this])
-
-      (^void uploadChunk [this ^String location ^ByteBuffer chunk])
-
-      (^void finishFileUpload [this ^String location])
-
-      (^String beginFileDownload [this ^String file])
-
-      (^ByteBuffer downloadChunk [this ^String id])
-
-      (^String getNimbusConf [this])
+(defn dummy-service-handler 
+  ([conf inimbus auth-context]
+     (let [nimbus-d (nimbus-data conf inimbus)
+           topo-conf (atom nil)]
+       (reify Nimbus$Iface
+         (^void submitTopologyWithOpts [this ^String storm-name ^String uploadedJarLocation ^String serializedConf ^StormTopology topology
+                                        ^SubmitOptions submitOptions]
+           (if (not (nil? serializedConf)) (swap! topo-conf (fn [prev new] new) (from-json serializedConf)))
+           (nimbus/check-authorization! nimbus-d storm-name @topo-conf "submitTopology" auth-context))
+         
+         (^void killTopology [this ^String storm-name]
+           (nimbus/check-authorization! nimbus-d storm-name @topo-conf "killTopology" auth-context))
+         
+         (^void killTopologyWithOpts [this ^String storm-name ^KillOptions options]
+           (nimbus/check-authorization! nimbus-d storm-name @topo-conf "killTopology" auth-context))
+         
+         (^void rebalance [this ^String storm-name ^RebalanceOptions options]
+           (nimbus/check-authorization! nimbus-d storm-name @topo-conf "rebalance" auth-context))
+         
+         (activate [this storm-name]
+           (nimbus/check-authorization! nimbus-d storm-name @topo-conf "activate" auth-context))
+         
+         (deactivate [this storm-name]
+           (nimbus/check-authorization! nimbus-d storm-name @topo-conf "deactivate" auth-context))
 
-      (^String getTopologyConf [this ^String id])
+         (uploadNewCredentials [this storm-name creds]
+           (nimbus/check-authorization! nimbus-d storm-name @topo-conf "uploadNewCredentials" auth-context)) 
+         
+         (beginFileUpload [this])
+         
+         (^void uploadChunk [this ^String location ^ByteBuffer chunk])
+         
+         (^void finishFileUpload [this ^String location])
+         
+         (^String beginFileDownload [this ^String file]
+           (nimbus/check-authorization! nimbus-d nil nil "fileDownload" auth-context)
+           "Done!")
+         
+         (^ByteBuffer downloadChunk [this ^String id])
+         
+         (^String getNimbusConf [this])
+         
+         (^String getTopologyConf [this ^String id])
+         
+         (^StormTopology getTopology [this ^String id])
+         
+         (^StormTopology getUserTopology [this ^String id])
+         
+         (^ClusterSummary getClusterInfo [this])
+         
+         (^TopologyInfo getTopologyInfo [this ^String storm-id]))))
+  ([conf inimbus]
+     (dummy-service-handler conf inimbus nil)))
+     
 
-      (^StormTopology getTopology [this ^String id])
-
-      (^StormTopology getUserTopology [this ^String id])
-
-      (^ClusterSummary getClusterInfo [this])
-      
-      (^TopologyInfo getTopologyInfo [this ^String storm-id]))))
-
-(defn launch-server [server-port login-cfg aznClass transportPluginClass] 
+(defn launch-server [server-port login-cfg aznClass transportPluginClass serverConf] 
   (let [conf1 (merge (read-storm-config)
-                    {NIMBUS-AUTHORIZER aznClass 
-                     NIMBUS-HOST "localhost"
-                     NIMBUS-THRIFT-PORT server-port
-                     STORM-THRIFT-TRANSPORT-PLUGIN transportPluginClass})
-        conf (if login-cfg (merge conf1 {"java.security.auth.login.config" login-cfg}) conf1)
+                     {NIMBUS-AUTHORIZER aznClass 
+                      NIMBUS-HOST "localhost"
+                      NIMBUS-THRIFT-PORT server-port
+                      STORM-THRIFT-TRANSPORT-PLUGIN transportPluginClass})
+        conf2 (if login-cfg (merge conf1 {"java.security.auth.login.config" login-cfg}) conf1)
+        conf (if serverConf (merge conf2 serverConf) conf2)
         nimbus (nimbus/standalone-nimbus)
         service-handler (dummy-service-handler conf nimbus)
-        server (ThriftServer. conf (Nimbus$Processor. service-handler) (int (conf NIMBUS-THRIFT-PORT)))]
+        server (ThriftServer. 
+                conf 
+                (Nimbus$Processor. service-handler) 
+                ThriftConnectionType/NIMBUS)]
     (.addShutdownHook (Runtime/getRuntime) (Thread. (fn [] (.stop server))))
     (.start (Thread. #(.serve server)))
     (wait-for-condition #(.isServing server))
@@ -133,32 +141,172 @@
 
 (defmacro with-server [args & body]
   `(let [server# (launch-server ~@args)]
-      ~@body
-      (.stop server#)
-      ))
+     ~@body
+     (.stop server#)
+     ))
 
-(deftest Simple-authentication-test 
+(deftest kerb-to-local-test
+  (let [kptol (KerberosPrincipalToLocal. )]
+    (.prepare kptol {})
+    (is (= "me" (.toLocal kptol (mk-principal "me@realm"))))
+    (is (= "simple" (.toLocal kptol (mk-principal "simple"))))
+    (is (= "someone" (.toLocal kptol (mk-principal "someone/host@realm"))))))
+
+(deftest Simple-authentication-test
   (let [a-port (available-port)]
-    (with-server [a-port nil nil "backtype.storm.security.auth.SimpleTransportPlugin"]
+    (with-server [a-port nil nil "backtype.storm.security.auth.SimpleTransportPlugin" nil]
       (let [storm-conf (merge (read-storm-config)
                               {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin"})
             client (NimbusClient. storm-conf "localhost" a-port nimbus-timeout)
             nimbus_client (.getClient client)]
         (.activate nimbus_client "security_auth_test_topology")
         (.close client))
-
+      
       (let [storm-conf (merge (read-storm-config)
                               {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"
-                               "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest.conf"})]
+                               "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest.conf"
+                               STORM-NIMBUS-RETRY-TIMES 0})]
         (testing "(Negative authentication) Server: Simple vs. Client: Digest"
           (is (thrown-cause?  java.net.SocketTimeoutException
                               (NimbusClient. storm-conf "localhost" a-port nimbus-timeout))))))))
   
+(deftest negative-whitelist-authorization-test 
+  (let [a-port (available-port)]
+    (with-server [a-port nil
+                  "backtype.storm.security.auth.authorizer.SimpleWhitelistAuthorizer" 
+                  "backtype.storm.testing.SingleUserSimpleTransport" nil]
+      (let [storm-conf (merge (read-storm-config)
+                              {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.testing.SingleUserSimpleTransport"})
+            client (NimbusClient. storm-conf "localhost" a-port nimbus-timeout)
+            nimbus_client (.getClient client)]
+        (testing "(Negative authorization) Authorization plugin should reject client request"
+          (is (thrown-cause? AuthorizationException
+                             (.activate nimbus_client "security_auth_test_topology"))))
+        (.close client)))))
+
+(deftest positive-whitelist-authorization-test 
+    (let [a-port (available-port)]
+      (with-server [a-port nil
+                    "backtype.storm.security.auth.authorizer.SimpleWhitelistAuthorizer" 
+                    "backtype.storm.testing.SingleUserSimpleTransport" {SimpleWhitelistAuthorizer/WHITELIST_USERS_CONF ["user"]}]
+        (let [storm-conf (merge (read-storm-config)
+                                {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.testing.SingleUserSimpleTransport"})
+              client (NimbusClient. storm-conf "localhost" a-port nimbus-timeout)
+              nimbus_client (.getClient client)]
+          (testing "(Positive authorization) Authorization plugin should accept client request"
+            (.activate nimbus_client "security_auth_test_topology"))
+          (.close client)))))
+
+(deftest simple-acl-user-auth-test
+  (let [cluster-conf (merge (read-storm-config)
+                       {NIMBUS-ADMINS ["admin"]
+                        NIMBUS-SUPERVISOR-USERS ["supervisor"]})
+        authorizer (SimpleACLAuthorizer. )
+        admin-user (mk-subject "admin")
+        supervisor-user (mk-subject "supervisor")
+        user-a (mk-subject "user-a")
+        user-b (mk-subject "user-b")]
+  (.prepare authorizer cluster-conf) 
+  (is (= true (.permit authorizer (ReqContext. user-a) "submitTopology" {})))
+  (is (= true (.permit authorizer (ReqContext. user-b) "submitTopology" {})))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "submitTopology" {})))
+  (is (= false (.permit authorizer (ReqContext. supervisor-user) "submitTopology" {})))
+
+  (is (= true (.permit authorizer (ReqContext. user-a) "fileUpload" nil)))
+  (is (= true (.permit authorizer (ReqContext. user-b) "fileUpload" nil)))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "fileUpload" nil)))
+  (is (= false (.permit authorizer (ReqContext. supervisor-user) "fileUpload" nil)))
+
+  (is (= true (.permit authorizer (ReqContext. user-a) "getNimbusConf" nil)))
+  (is (= true (.permit authorizer (ReqContext. user-b) "getNimbusConf" nil)))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "getNimbusConf" nil)))
+  (is (= false (.permit authorizer (ReqContext. supervisor-user) "getNimbusConf" nil)))
+
+  (is (= true (.permit authorizer (ReqContext. user-a) "getClusterInfo" nil)))
+  (is (= true (.permit authorizer (ReqContext. user-b) "getClusterInfo" nil)))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "getClusterInfo" nil)))
+  (is (= false (.permit authorizer (ReqContext. supervisor-user) "getClusterInfo" nil)))
+
+  (is (= false (.permit authorizer (ReqContext. user-a) "fileDownload" nil)))
+  (is (= false (.permit authorizer (ReqContext. user-b) "fileDownload" nil)))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "fileDownload" nil)))
+  (is (= true (.permit authorizer (ReqContext. supervisor-user) "fileDownload" nil)))
+
+  (is (= true (.permit authorizer (ReqContext. user-a) "killTopology" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= false (.permit authorizer (ReqContext. user-b) "killTopology" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "killTopology" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= false (.permit authorizer (ReqContext. supervisor-user) "killTopolgy" {TOPOLOGY-USERS ["user-a"]})))
+
+  (is (= true (.permit authorizer (ReqContext. user-a) "uploadNewCredentials" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= false (.permit authorizer (ReqContext. user-b) "uploadNewCredentials" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "uploadNewCredentials" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= false (.permit authorizer (ReqContext. supervisor-user) "uploadNewCredentials" {TOPOLOGY-USERS ["user-a"]})))
+
+  (is (= true (.permit authorizer (ReqContext. user-a) "rebalance" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= false (.permit authorizer (ReqContext. user-b) "rebalance" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "rebalance" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= false (.permit authorizer (ReqContext. supervisor-user) "rebalance" {TOPOLOGY-USERS ["user-a"]})))
+
+  (is (= true (.permit authorizer (ReqContext. user-a) "activate" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= false (.permit authorizer (ReqContext. user-b) "activate" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "activate" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= false (.permit authorizer (ReqContext. supervisor-user) "activate" {TOPOLOGY-USERS ["user-a"]})))
+
+  (is (= true (.permit authorizer (ReqContext. user-a) "deactivate" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= false (.permit authorizer (ReqContext. user-b) "deactivate" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "deactivate" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= false (.permit authorizer (ReqContext. supervisor-user) "deactivate" {TOPOLOGY-USERS ["user-a"]})))
+
+  (is (= true (.permit authorizer (ReqContext. user-a) "getTopologyConf" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= false (.permit authorizer (ReqContext. user-b) "getTopologyConf" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "getTopologyConf" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= false (.permit authorizer (ReqContext. supervisor-user) "getTopologyConf" {TOPOLOGY-USERS ["user-a"]})))
+
+  (is (= true (.permit authorizer (ReqContext. user-a) "getTopology" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= false (.permit authorizer (ReqContext. user-b) "getTopology" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "getTopology" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= false (.permit authorizer (ReqContext. supervisor-user) "getTopology" {TOPOLOGY-USERS ["user-a"]})))
+
+  (is (= true (.permit authorizer (ReqContext. user-a) "getUserTopology" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= false (.permit authorizer (ReqContext. user-b) "getUserTopology" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "getUserTopology" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= false (.permit authorizer (ReqContext. supervisor-user) "getUserTopology" {TOPOLOGY-USERS ["user-a"]})))
+
+  (is (= true (.permit authorizer (ReqContext. user-a) "getTopologyInfo" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= false (.permit authorizer (ReqContext. user-b) "getTopologyInfo" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "getTopologyInfo" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= false (.permit authorizer (ReqContext. supervisor-user) "getTopologyInfo" {TOPOLOGY-USERS ["user-a"]})))
+))
+
+(deftest simple-acl-same-user-auth-test
+  (let [cluster-conf (merge (read-storm-config)
+                       {NIMBUS-ADMINS ["admin"]
+                        NIMBUS-SUPERVISOR-USERS ["admin"]})
+        authorizer (SimpleACLAuthorizer. )
+        admin-user (mk-subject "admin")]
+  (.prepare authorizer cluster-conf) 
+  (is (= true (.permit authorizer (ReqContext. admin-user) "submitTopology" {})))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "fileUpload" nil)))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "getNimbusConf" nil)))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "getClusterInfo" nil)))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "fileDownload" nil)))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "killTopology" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "uploadNewCredentials" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "rebalance" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "activate" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "deactivate" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "getTopologyConf" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "getTopology" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "getUserTopology" {TOPOLOGY-USERS ["user-a"]})))
+  (is (= true (.permit authorizer (ReqContext. admin-user) "getTopologyInfo" {TOPOLOGY-USERS ["user-a"]})))
+))
+
+
 (deftest positive-authorization-test 
   (let [a-port (available-port)]
-    (with-server [a-port nil 
+    (with-server [a-port nil
                   "backtype.storm.security.auth.authorizer.NoopAuthorizer" 
-                  "backtype.storm.security.auth.SimpleTransportPlugin"]
+                  "backtype.storm.security.auth.SimpleTransportPlugin" nil]
       (let [storm-conf (merge (read-storm-config)
                               {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin"})
             client (NimbusClient. storm-conf "localhost" a-port nimbus-timeout)
@@ -171,7 +319,7 @@
   (let [a-port (available-port)]
     (with-server [a-port nil
                   "backtype.storm.security.auth.authorizer.DenyAuthorizer" 
-                  "backtype.storm.security.auth.SimpleTransportPlugin"]
+                  "backtype.storm.security.auth.SimpleTransportPlugin" nil]
       (let [storm-conf (merge (read-storm-config)
                               {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin"
                                Config/NIMBUS_HOST "localhost"
@@ -180,19 +328,20 @@
             client (NimbusClient/getConfiguredClient storm-conf)
             nimbus_client (.getClient client)]
         (testing "(Negative authorization) Authorization plugin should reject client request"
-          (is (thrown? TTransportException
-                       (.activate nimbus_client "security_auth_test_topology"))))
+          (is (thrown-cause? AuthorizationException
+                             (.activate nimbus_client "security_auth_test_topology"))))
         (.close client)))))
 
 (deftest digest-authentication-test
   (let [a-port (available-port)]
-    (with-server [a-port  
+    (with-server [a-port
                   "test/clj/backtype/storm/security/auth/jaas_digest.conf" 
                   nil
-                  "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"]
+                  "backtype.storm.security.auth.digest.DigestSaslTransportPlugin" nil]
       (let [storm-conf (merge (read-storm-config)
                               {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"
-                               "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest.conf"})
+                               "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest.conf"
+                               STORM-NIMBUS-RETRY-TIMES 0})
             client (NimbusClient. storm-conf "localhost" a-port nimbus-timeout)
             nimbus_client (.getClient client)]
         (testing "(Positive authentication) valid digest authentication"
@@ -200,7 +349,8 @@
         (.close client))
       
       (let [storm-conf (merge (read-storm-config)
-                              {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin"})
+                              {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin"
+                               STORM-NIMBUS-RETRY-TIMES 0})
             client (NimbusClient. storm-conf "localhost" a-port nimbus-timeout)
             nimbus_client (.getClient client)]
         (testing "(Negative authentication) Server: Digest vs. Client: Simple"
@@ -210,34 +360,37 @@
       
       (let [storm-conf (merge (read-storm-config)
                               {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"
-                             "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest_bad_password.conf"})]
+                               "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest_bad_password.conf"
+                               STORM-NIMBUS-RETRY-TIMES 0})]
         (testing "(Negative authentication) Invalid  password"
-          (is (thrown? TTransportException
-                       (NimbusClient. storm-conf "localhost" a-port nimbus-timeout)))))
+          (is (thrown-cause? TTransportException
+                             (NimbusClient. storm-conf "localhost" a-port nimbus-timeout)))))
       
       (let [storm-conf (merge (read-storm-config)
                               {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"
-                               "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest_unknown_user.conf"})]
+                               "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest_unknown_user.conf"
+                               STORM-NIMBUS-RETRY-TIMES 0})]
         (testing "(Negative authentication) Unknown user"
-          (is (thrown? TTransportException 
-                       (NimbusClient. storm-conf "localhost" a-port nimbus-timeout)))))
-      
+          (is (thrown-cause? TTransportException 
+                             (NimbusClient. storm-conf "localhost" a-port nimbus-timeout)))))
+
       (let [storm-conf (merge (read-storm-config)
                               {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"
-                               "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/nonexistent.conf"})]
+                               "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/nonexistent.conf"
+                               STORM-NIMBUS-RETRY-TIMES 0})]
         (testing "(Negative authentication) nonexistent configuration file"
-          (is (thrown? RuntimeException 
-                       (NimbusClient. storm-conf "localhost" a-port nimbus-timeout)))))
+          (is (thrown-cause? RuntimeException 
+                             (NimbusClient. storm-conf "localhost" a-port nimbus-timeout)))))
       
       (let [storm-conf (merge (read-storm-config)
                               {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"
-                               "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest_missing_client.conf"})]
+                               "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest_missing_client.conf"
+                               STORM-NIMBUS-RETRY-TIMES 0})]
         (testing "(Negative authentication) Missing client"
           (is (thrown-cause? java.io.IOException
                              (NimbusClient. storm-conf "localhost" a-port nimbus-timeout))))))))
-  
-  
+        
 (deftest test-GetTransportPlugin-throws-RuntimeException
   (let [conf (merge (read-storm-config)
                     {Config/STORM_THRIFT_TRANSPORT_PLUGIN "null.invalid"})]
-    (is (thrown? RuntimeException (AuthUtils/GetTransportPlugin conf nil)))))
+    (is (thrown-cause? RuntimeException (AuthUtils/GetTransportPlugin conf nil nil)))))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/backtype/storm/security/auth/authorizer/DRPCSimpleAclAuthorizer_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/security/auth/authorizer/DRPCSimpleAclAuthorizer_test.clj b/storm-core/test/clj/backtype/storm/security/auth/authorizer/DRPCSimpleAclAuthorizer_test.clj
new file mode 100644
index 0000000..d592a40
--- /dev/null
+++ b/storm-core/test/clj/backtype/storm/security/auth/authorizer/DRPCSimpleAclAuthorizer_test.clj
@@ -0,0 +1,226 @@
+(ns backtype.storm.security.auth.authorizer.DRPCSimpleAclAuthorizer-test
+  (:use [clojure test])
+  (:import [org.mockito Mockito])
+  (:import [backtype.storm Config])
+  (:import [backtype.storm.security.auth ReqContext SingleUserPrincipal])
+  (:import [backtype.storm.security.auth.authorizer DRPCSimpleACLAuthorizer])
+  (:use [backtype.storm config util])
+  )
+
+(defn- mk-mock-context [user]
+  (let [mock-context (Mockito/mock ReqContext)]
+    (. (Mockito/when (.principal mock-context)) thenReturn
+       (SingleUserPrincipal. user))
+    mock-context))
+
+(let [function "jump"
+      partial-function "partial"
+      alice-context (mk-mock-context "alice")
+      alice-kerb-context (mk-mock-context "alice@SOME.RELM")
+      bob-context (mk-mock-context "bob")
+      charlie-context (mk-mock-context "charlie")
+      acl-file "drpc-simple-acl-test-scenario.yaml"
+      strict-handler (doto (DRPCSimpleACLAuthorizer.)
+                           (.prepare {DRPC-AUTHORIZER-ACL-STRICT true
+                                      DRPC-AUTHORIZER-ACL-FILENAME acl-file
+                                      STORM-PRINCIPAL-TO-LOCAL-PLUGIN "backtype.storm.security.auth.KerberosPrincipalToLocal"}))
+      permissive-handler (doto (DRPCSimpleACLAuthorizer.)
+                               (.prepare {DRPC-AUTHORIZER-ACL-STRICT false
+                                          DRPC-AUTHORIZER-ACL-FILENAME acl-file
+                                          STORM-PRINCIPAL-TO-LOCAL-PLUGIN "backtype.storm.security.auth.KerberosPrincipalToLocal"}))]
+
+  (deftest test-partial-authorization
+    (testing "deny execute to unauthorized user"
+      (is (not
+            (.permit strict-handler
+                     (ReqContext/context)
+                     "execute"
+                     {DRPCSimpleACLAuthorizer/FUNCTION_KEY partial-function}))))
+
+    (testing "allow execute to authorized kerb user for correct function"
+      (is
+        (.permit
+          strict-handler
+          alice-kerb-context
+          "execute"
+          {DRPCSimpleACLAuthorizer/FUNCTION_KEY partial-function})))
+
+      (testing "deny fetchRequest to unauthorized user for correct function"
+        (is (not
+              (.permit
+                strict-handler
+                alice-kerb-context
+                "fetchRequest"
+                {DRPCSimpleACLAuthorizer/FUNCTION_KEY partial-function}))))
+    )
+
+  (deftest test-client-authorization-strict
+    (testing "deny execute to unauthorized user"
+      (is (not
+            (.permit strict-handler
+                     (ReqContext/context)
+                     "execute"
+                     {DRPCSimpleACLAuthorizer/FUNCTION_KEY function}))))
+
+    (testing "deny execute to valid user for incorrect function"
+      (is (not
+            (.permit
+              strict-handler
+              alice-context
+              "execute"
+              {DRPCSimpleACLAuthorizer/FUNCTION_KEY "wrongFunction"}))))
+
+    (testing "allow execute to authorized kerb user for correct function"
+      (is
+        (.permit
+          strict-handler
+          alice-kerb-context
+          "execute"
+          {DRPCSimpleACLAuthorizer/FUNCTION_KEY function})))
+
+    (testing "allow execute to authorized user for correct function"
+      (is
+        (.permit
+          strict-handler
+          alice-context
+          "execute"
+          {DRPCSimpleACLAuthorizer/FUNCTION_KEY function}))))
+
+
+  (deftest test-client-authorization-permissive
+    (testing "deny execute to unauthorized user for correct function"
+      (is (not
+            (.permit permissive-handler
+                     (ReqContext/context)
+                     "execute"
+                     {DRPCSimpleACLAuthorizer/FUNCTION_KEY function}))))
+
+    (testing "allow execute for user for incorrect function when permissive"
+      (is
+        (.permit permissive-handler
+                 alice-context
+                 "execute"
+                 {DRPCSimpleACLAuthorizer/FUNCTION_KEY "wrongFunction"})))
+
+    (testing "allow execute for user for incorrect function when permissive"
+      (is
+        (.permit permissive-handler
+                 alice-kerb-context
+                 "execute"
+                 {DRPCSimpleACLAuthorizer/FUNCTION_KEY "wrongFunction"})))
+
+    (testing "allow execute to authorized user for correct function"
+      (is
+        (.permit permissive-handler
+                 bob-context
+                 "execute"
+                 {DRPCSimpleACLAuthorizer/FUNCTION_KEY function}))))
+
+  (deftest test-invocation-authorization-strict
+    (doseq [operation ["fetchRequest" "failRequest" "result"]]
+
+      (testing (str "deny " operation
+                    " to unauthorized user for correct function")
+        (is (not
+              (.permit
+                strict-handler
+                alice-context
+                operation
+                {DRPCSimpleACLAuthorizer/FUNCTION_KEY function})))
+
+      (testing (str "deny " operation
+                    " to user for incorrect function when strict")
+        (is (not
+              (.permit
+                strict-handler
+                charlie-context
+                operation
+                {DRPCSimpleACLAuthorizer/FUNCTION_KEY "wrongFunction"}))))
+
+      (testing (str "allow " operation
+                    " to authorized user for correct function")
+        (is
+          (.permit
+            strict-handler
+            charlie-context
+            operation
+            {DRPCSimpleACLAuthorizer/FUNCTION_KEY function}))))))
+
+  (deftest test-invocation-authorization-permissive
+    (doseq [operation ["fetchRequest" "failRequest" "result"]]
+
+      (testing (str "deny " operation
+                    " to unauthorized user for correct function")
+        (is (not
+              (.permit
+                permissive-handler
+                bob-context
+                operation
+                {DRPCSimpleACLAuthorizer/FUNCTION_KEY function}))))
+
+        (testing (str "allow " operation
+                      " to user for incorrect function when permissive")
+          (is
+            (.permit
+              permissive-handler
+              charlie-context
+              operation
+              {DRPCSimpleACLAuthorizer/FUNCTION_KEY "wrongFunction"})))
+
+      (testing (str operation " is allowed for authorized user")
+        (is
+          (.permit
+            permissive-handler
+            charlie-context
+            operation
+            {DRPCSimpleACLAuthorizer/FUNCTION_KEY function})))))
+
+  (deftest test-deny-when-no-function-given
+    (is (not
+         (.permit strict-handler alice-context "execute" {})))
+
+    (is (not
+         (.permit
+           strict-handler
+           alice-context
+           "execute"
+           {DRPCSimpleACLAuthorizer/FUNCTION_KEY nil})))
+
+    (is (not
+         (.permit permissive-handler bob-context "execute" {})))
+
+    (is (not
+         (.permit
+           permissive-handler
+           bob-context
+           "execute"
+           {DRPCSimpleACLAuthorizer/FUNCTION_KEY nil}))))
+
+  (deftest test-deny-when-invalid-user-given
+    (is (not
+          (.permit
+            strict-handler
+            (Mockito/mock ReqContext)
+            "execute"
+            {DRPCSimpleACLAuthorizer/FUNCTION_KEY function})))
+
+    (is (not
+          (.permit
+            strict-handler
+            nil
+            "execute"
+            {DRPCSimpleACLAuthorizer/FUNCTION_KEY function})))
+
+    (is (not
+          (.permit
+            permissive-handler
+            (Mockito/mock ReqContext)
+            "execute"
+            {DRPCSimpleACLAuthorizer/FUNCTION_KEY function})))
+
+    (is (not
+          (.permit
+            permissive-handler
+            nil
+            "execute"
+            {DRPCSimpleACLAuthorizer/FUNCTION_KEY function})))))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/backtype/storm/security/auth/auto_login_module_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/security/auth/auto_login_module_test.clj b/storm-core/test/clj/backtype/storm/security/auth/auto_login_module_test.clj
new file mode 100644
index 0000000..2056509
--- /dev/null
+++ b/storm-core/test/clj/backtype/storm/security/auth/auto_login_module_test.clj
@@ -0,0 +1,91 @@
+;; 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.security.auth.auto-login-module-test
+  (:use [clojure test])
+  (:use [backtype.storm util])
+  (:import [backtype.storm.security.auth.kerberos AutoTGT
+            AutoTGTKrb5LoginModule AutoTGTKrb5LoginModuleTest])
+  (:import [javax.security.auth Subject Subject])
+  (:import [javax.security.auth.kerberos KerberosTicket])
+  (:import [org.mockito Mockito])
+  )
+
+(deftest login-module-no-subj-no-tgt-test
+  (testing "Behavior is correct when there is no Subject or TGT"
+    (let [login-module (AutoTGTKrb5LoginModule.)]
+
+      (is (thrown-cause? javax.security.auth.login.LoginException
+                         (.login login-module)))
+      (is (not (.commit login-module)))
+      (is (not (.abort login-module)))
+      (is (.logout login-module)))))
+
+(deftest login-module-readonly-subj-no-tgt-test
+  (testing "Behavior is correct when there is a read-only Subject and no TGT"
+    (let [readonly-subj (Subject. true #{} #{} #{})
+          login-module (AutoTGTKrb5LoginModule.)]
+      (.initialize login-module readonly-subj nil nil nil)
+      (is (not (.commit login-module)))
+      (is (.logout login-module)))))
+
+(deftest login-module-with-subj-no-tgt-test
+  (testing "Behavior is correct when there is a Subject and no TGT"
+    (let [login-module (AutoTGTKrb5LoginModule.)]
+      (.initialize login-module (Subject.) nil nil nil)
+      (is (thrown-cause? javax.security.auth.login.LoginException
+                         (.login login-module)))
+      (is (not (.commit login-module)))
+      (is (not (.abort login-module)))
+      (is (.logout login-module)))))
+
+(deftest login-module-no-subj-with-tgt-test
+  (testing "Behavior is correct when there is no Subject and a TGT"
+    (let [login-module (AutoTGTKrb5LoginModuleTest.)]
+      (.setKerbTicket login-module (Mockito/mock KerberosTicket))
+      (is (.login login-module))
+      (is (thrown-cause? javax.security.auth.login.LoginException
+                         (.commit login-module)))
+
+      (.setKerbTicket login-module (Mockito/mock KerberosTicket))
+      (is (.abort login-module))
+      (is (.logout login-module)))))
+
+(deftest login-module-readonly-subj-with-tgt-test
+  (testing "Behavior is correct when there is a read-only Subject and a TGT"
+    (let [readonly-subj (Subject. true #{} #{} #{})
+          login-module (AutoTGTKrb5LoginModuleTest.)]
+      (.initialize login-module readonly-subj nil nil nil)
+      (.setKerbTicket login-module (Mockito/mock KerberosTicket))
+      (is (.login login-module))
+      (is (thrown-cause? javax.security.auth.login.LoginException
+                         (.commit login-module)))
+
+      (.setKerbTicket login-module (Mockito/mock KerberosTicket))
+      (is (.abort login-module))
+      (is (.logout login-module)))))
+
+(deftest login-module-with-subj-and-tgt
+  (testing "Behavior is correct when there is a Subject and a TGT"
+    (let [login-module (AutoTGTKrb5LoginModuleTest.)
+          _ (set! (. login-module client) (Mockito/mock
+                                            java.security.Principal))
+          ticket (Mockito/mock KerberosTicket)]
+      (.initialize login-module (Subject.) nil nil nil)
+      (.setKerbTicket login-module ticket)
+      (is (.login login-module))
+      (is (.commit login-module))
+      (is (.abort login-module))
+      (is (.logout login-module)))))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-alice.jaas
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-alice.jaas b/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-alice.jaas
new file mode 100644
index 0000000..cd691ae
--- /dev/null
+++ b/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-alice.jaas
@@ -0,0 +1,5 @@
+StormClient {
+       org.apache.zookeeper.server.auth.DigestLoginModule required
+       username="alice"
+       password="poorpasswordforalice";
+};

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-bob.jaas
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-bob.jaas b/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-bob.jaas
new file mode 100644
index 0000000..e4ca097
--- /dev/null
+++ b/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-bob.jaas
@@ -0,0 +1,5 @@
+StormClient {
+       org.apache.zookeeper.server.auth.DigestLoginModule required
+       username="bob"
+       password="poorpasswordforbob";
+};

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-charlie.jaas
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-charlie.jaas b/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-charlie.jaas
new file mode 100644
index 0000000..3473d6d
--- /dev/null
+++ b/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-charlie.jaas
@@ -0,0 +1,5 @@
+StormClient {
+       org.apache.zookeeper.server.auth.DigestLoginModule required
+       username="charlie"
+       password="poorpasswordforcharlie";
+};

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-server.jaas
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-server.jaas b/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-server.jaas
new file mode 100644
index 0000000..3b22d21
--- /dev/null
+++ b/storm-core/test/clj/backtype/storm/security/auth/drpc-auth-server.jaas
@@ -0,0 +1,6 @@
+StormServer {
+       org.apache.zookeeper.server.auth.DigestLoginModule required
+       user_alice="poorpasswordforalice"
+       user_bob="poorpasswordforbob"
+       user_charlie="poorpasswordforcharlie";
+};

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/backtype/storm/security/auth/drpc_auth_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/security/auth/drpc_auth_test.clj b/storm-core/test/clj/backtype/storm/security/auth/drpc_auth_test.clj
new file mode 100644
index 0000000..ff431ec
--- /dev/null
+++ b/storm-core/test/clj/backtype/storm/security/auth/drpc_auth_test.clj
@@ -0,0 +1,315 @@
+;; 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.security.auth.drpc-auth-test
+  (:use [clojure test])
+  (:require [backtype.storm.daemon [drpc :as drpc]])
+  (:import [backtype.storm.generated AuthorizationException
+            DRPCExecutionException DistributedRPC$Processor
+            DistributedRPCInvocations$Processor])
+  (:import [backtype.storm Config])
+  (:import [backtype.storm.security.auth ReqContext SingleUserPrincipal ThriftServer ThriftConnectionType])
+  (:import [backtype.storm.utils DRPCClient])
+  (:import [backtype.storm.drpc DRPCInvocationsClient])
+  (:import [java.util.concurrent TimeUnit])
+  (:import [javax.security.auth Subject])
+  (:use [backtype.storm bootstrap util])
+  (:use [backtype.storm.daemon common])
+  (:use [backtype.storm bootstrap testing])
+  )
+
+(bootstrap)
+
+(def drpc-timeout (Integer. 30))
+
+(defn launch-server [conf drpcAznClass transportPluginClass login-cfg client-port invocations-port]
+  (let [conf (if drpcAznClass (assoc conf DRPC-AUTHORIZER drpcAznClass) conf)
+        conf (if transportPluginClass (assoc conf STORM-THRIFT-TRANSPORT-PLUGIN transportPluginClass) conf)
+        conf (if login-cfg (assoc conf "java.security.auth.login.config" login-cfg) conf)
+        conf (assoc conf DRPC-PORT client-port)
+        conf (assoc conf DRPC-INVOCATIONS-PORT invocations-port)
+        service-handler (drpc/service-handler conf)
+        handler-server (ThriftServer. conf
+                                      (DistributedRPC$Processor. service-handler)
+                                      ThriftConnectionType/DRPC)
+        invoke-server (ThriftServer. conf
+                                     (DistributedRPCInvocations$Processor. service-handler)
+                                     ThriftConnectionType/DRPC_INVOCATIONS)]
+    (.addShutdownHook (Runtime/getRuntime) (Thread. (fn [] (.stop handler-server) (.stop invoke-server))))
+    (log-message "storm conf:" conf)
+    (log-message "Starting DRPC invocation server ...")
+    (.start (Thread. #(.serve invoke-server)))
+    (wait-for-condition #(.isServing invoke-server))
+    (log-message "Starting DRPC handler server ...")
+    (.start (Thread. #(.serve handler-server)))
+    (wait-for-condition #(.isServing handler-server))
+    [handler-server invoke-server]))
+
+(defmacro with-server [args & body]
+  `(let [[handler-server# invoke-server#] (launch-server ~@args)]
+      ~@body
+      (log-message "Stopping DRPC servers ...")
+      (.stop handler-server#)
+      (.stop invoke-server#)
+      ))
+
+(deftest deny-drpc-test
+  (let [client-port (available-port)
+        invocations-port (available-port (inc client-port))
+        storm-conf (read-storm-config)]
+    (with-server [storm-conf "backtype.storm.security.auth.authorizer.DenyAuthorizer"
+                  nil nil client-port invocations-port]
+      (let [drpc (DRPCClient. storm-conf "localhost" client-port)
+            drpc_client (.getClient drpc)
+            invocations (DRPCInvocationsClient. storm-conf "localhost" invocations-port)
+            invocations_client (.getClient invocations)]
+        (is (thrown? AuthorizationException (.execute drpc_client "func-foo" "args-bar")))
+        (is (thrown? AuthorizationException (.fetchRequest invocations_client nil)))
+        (.close drpc)
+        (.close invocations)))))
+
+(deftest deny-drpc-digest-test
+  (let [client-port (available-port)
+        invocations-port (available-port (inc client-port))
+        storm-conf (read-storm-config)]
+    (with-server [storm-conf "backtype.storm.security.auth.authorizer.DenyAuthorizer"
+                  "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"
+                  "test/clj/backtype/storm/security/auth/jaas_digest.conf"
+                  client-port invocations-port]
+      (let [conf (merge storm-conf {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"
+                             "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest.conf"})
+            drpc (DRPCClient. conf "localhost" client-port)
+            drpc_client (.getClient drpc)
+            invocations (DRPCInvocationsClient. conf "localhost" invocations-port)
+            invocations_client (.getClient invocations)]
+        (is (thrown? AuthorizationException (.execute drpc_client "func-foo" "args-bar")))
+        (is (thrown? AuthorizationException (.fetchRequest invocations_client nil)))
+        (.close drpc)
+        (.close invocations)))))
+
+(defmacro with-simple-drpc-test-scenario
+  [[strict? alice-client bob-client charlie-client alice-invok charlie-invok] & body]
+  (let [client-port (available-port)
+        invocations-port (available-port (inc client-port))
+        storm-conf (merge (read-storm-config)
+                          {DRPC-AUTHORIZER-ACL-STRICT strict?
+                           DRPC-AUTHORIZER-ACL-FILENAME "drpc-simple-acl-test-scenario.yaml"
+                           STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"})]
+    `(with-server [~storm-conf
+                   "backtype.storm.security.auth.authorizer.DRPCSimpleACLAuthorizer"
+                   "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"
+                   "test/clj/backtype/storm/security/auth/drpc-auth-server.jaas"
+                   ~client-port ~invocations-port]
+       (let [~alice-client (DRPCClient.
+                           (merge ~storm-conf {"java.security.auth.login.config"
+                                              "test/clj/backtype/storm/security/auth/drpc-auth-alice.jaas"})
+                           "localhost"
+                           ~client-port)
+             ~bob-client (DRPCClient.
+                         (merge ~storm-conf {"java.security.auth.login.config"
+                                            "test/clj/backtype/storm/security/auth/drpc-auth-bob.jaas"})
+                         "localhost"
+                         ~client-port)
+             ~charlie-client (DRPCClient.
+                               (merge ~storm-conf {"java.security.auth.login.config"
+                                                  "test/clj/backtype/storm/security/auth/drpc-auth-charlie.jaas"})
+                               "localhost"
+                               ~client-port)
+             ~alice-invok (DRPCInvocationsClient.
+                            (merge ~storm-conf {"java.security.auth.login.config"
+                                               "test/clj/backtype/storm/security/auth/drpc-auth-alice.jaas"})
+                            "localhost"
+                            ~invocations-port)
+             ~charlie-invok (DRPCInvocationsClient.
+                             (merge ~storm-conf {"java.security.auth.login.config"
+                                                "test/clj/backtype/storm/security/auth/drpc-auth-charlie.jaas"})
+                             "localhost"
+                             ~invocations-port)]
+         (try
+           ~@body
+           (finally
+             (.close ~alice-client)
+             (.close ~bob-client)
+             (.close ~charlie-client)
+             (.close ~alice-invok)
+             (.close ~charlie-invok)))))))
+
+(deftest drpc-per-function-auth-strict-test
+  (with-simple-drpc-test-scenario [true alice-client bob-client charlie-client alice-invok charlie-invok]
+    (let [drpc-timeout-seconds 10]
+      (testing "Permitted user can execute a function in the ACL"
+        (let [func "jump"
+              exec-ftr (future (.execute alice-client func "some args"))
+              id (atom "")
+              expected "Authorized DRPC"]
+          (with-timeout drpc-timeout-seconds TimeUnit/SECONDS
+            (while (empty? @id)
+              (reset! id
+                      (-> charlie-invok (.fetchRequest func) .get_request_id)))
+          (.result charlie-invok @id expected)
+          (is (= expected (.get exec-ftr drpc-timeout-seconds TimeUnit/SECONDS))))))
+
+      (testing "execute fails when function is not in ACL"
+        (is (thrown-cause? AuthorizationException
+          (.execute alice-client "jog" "some args"))))
+
+      (testing "fetchRequest fails when function is not in ACL"
+        (is (thrown-cause? AuthorizationException
+          (.fetchRequest charlie-invok "jog"))))
+
+      (testing "authorized user can fail a request"
+        (let [func "jump"
+              exec-ftr (future (.execute alice-client func "some args"))
+              id (atom "")]
+          (with-timeout drpc-timeout-seconds TimeUnit/SECONDS
+            (while (empty? @id)
+              (reset! id
+                      (-> charlie-invok (.fetchRequest func) .get_request_id)))
+          (.failRequest charlie-invok @id)
+          (is (thrown-cause? DRPCExecutionException
+                             (.get exec-ftr drpc-timeout-seconds TimeUnit/SECONDS))))))
+
+      (testing "unauthorized invocation user is denied returning a result"
+        (let [func "jump"
+              exec-ftr (future (.execute bob-client func "some args"))
+              id (atom "")
+              expected "Only Authorized User can populate the result"]
+          (with-timeout drpc-timeout-seconds TimeUnit/SECONDS
+            (while (empty? @id)
+              (reset! id
+                      (-> charlie-invok (.fetchRequest func) .get_request_id)))
+          (is (thrown-cause? AuthorizationException
+            (.result alice-invok @id "not the expected result")))
+          (.result charlie-invok @id expected)
+          (is (= expected (.get exec-ftr drpc-timeout-seconds TimeUnit/SECONDS))))))
+
+      (testing "unauthorized invocation user is denied failing a request"
+        (let [func "jump"
+              exec-ftr (future (.execute alice-client func "some args"))
+              id (atom "")]
+          (with-timeout drpc-timeout-seconds TimeUnit/SECONDS
+            (while (empty? @id)
+              (reset! id
+                      (-> charlie-invok (.fetchRequest func) .get_request_id)))
+          (is (thrown-cause? AuthorizationException
+            (.failRequest alice-invok @id)))
+          (.failRequest charlie-invok @id))))
+
+      (testing "unauthorized invocation user is denied fetching a request"
+        (let [func "jump"
+              exec-ftr (future (.execute bob-client func "some args"))
+              id (atom "")
+              expected "Only authorized users can fetchRequest"]
+          (Thread/sleep 1000)
+          (is (thrown-cause? AuthorizationException
+            (-> alice-invok (.fetchRequest func) .get_request_id)))
+          (with-timeout drpc-timeout-seconds TimeUnit/SECONDS
+            (while (empty? @id)
+              (reset! id
+                      (-> charlie-invok (.fetchRequest func) .get_request_id)))
+          (.result charlie-invok @id expected)
+          (is (= expected (.get exec-ftr drpc-timeout-seconds TimeUnit/SECONDS)))))))))
+
+(deftest drpc-per-function-auth-non-strict-test
+  (with-simple-drpc-test-scenario [false alice-client bob-client charlie-client alice-invok charlie-invok]
+    (let [drpc-timeout-seconds 10]
+      (testing "Permitted user can execute a function in the ACL"
+        (let [func "jump"
+              exec-ftr (future (.execute alice-client func "some args"))
+              id (atom "")
+              expected "Authorized DRPC"]
+          (with-timeout drpc-timeout-seconds TimeUnit/SECONDS
+            (while (empty? @id)
+              (reset! id
+                      (-> charlie-invok (.fetchRequest func) .get_request_id)))
+          (.result charlie-invok @id expected)
+          (is (= expected (.get exec-ftr drpc-timeout-seconds TimeUnit/SECONDS))))))
+
+      (testing "DRPC succeeds for anyone when function is not in ACL"
+        (let [func "jog"
+              exec-ftr (future (.execute charlie-client func "some args"))
+              id (atom "")
+              expected "Permissive/No ACL Entry"]
+          (with-timeout drpc-timeout-seconds TimeUnit/SECONDS
+            (while (empty? @id)
+              (reset! id
+                      (-> alice-invok (.fetchRequest func) .get_request_id)))
+          (.result alice-invok @id expected)
+          (is (= expected (.get exec-ftr drpc-timeout-seconds TimeUnit/SECONDS))))))
+
+      (testing "failure of a request is allowed when function is not in ACL"
+        (let [func "jog"
+              exec-ftr (future (.execute charlie-client func "some args"))
+              id (atom "")]
+          (with-timeout drpc-timeout-seconds TimeUnit/SECONDS
+            (while (empty? @id)
+              (reset! id
+                      (-> alice-invok (.fetchRequest func) .get_request_id)))
+          (.failRequest alice-invok @id)
+          (is (thrown-cause? DRPCExecutionException
+                             (.get exec-ftr drpc-timeout-seconds TimeUnit/SECONDS))))))
+
+      (testing "authorized user can fail a request"
+        (let [func "jump"
+              exec-ftr (future (.execute alice-client func "some args"))
+              id (atom "")]
+          (with-timeout drpc-timeout-seconds TimeUnit/SECONDS
+            (while (empty? @id)
+              (reset! id
+                      (-> charlie-invok (.fetchRequest func) .get_request_id)))
+          (.failRequest charlie-invok @id)
+          (is (thrown-cause? DRPCExecutionException
+                             (.get exec-ftr drpc-timeout-seconds TimeUnit/SECONDS))))))
+
+      (testing "unauthorized invocation user is denied returning a result"
+        (let [func "jump"
+              exec-ftr (future (.execute bob-client func "some args"))
+              id (atom "")
+              expected "Only Authorized User can populate the result"]
+          (with-timeout drpc-timeout-seconds TimeUnit/SECONDS
+            (while (empty? @id)
+              (reset! id
+                      (-> charlie-invok (.fetchRequest func) .get_request_id)))
+          (is (thrown-cause? AuthorizationException
+            (.result alice-invok @id "not the expected result")))
+          (.result charlie-invok @id expected)
+          (is (= expected (.get exec-ftr drpc-timeout-seconds TimeUnit/SECONDS))))))
+
+      (testing "unauthorized invocation user is denied failing a request"
+        (let [func "jump"
+              exec-ftr (future (.execute alice-client func "some args"))
+              id (atom "")]
+          (with-timeout drpc-timeout-seconds TimeUnit/SECONDS
+            (while (empty? @id)
+              (reset! id
+                      (-> charlie-invok (.fetchRequest func) .get_request_id)))
+          (is (thrown-cause? AuthorizationException
+            (.failRequest alice-invok @id)))
+          (.failRequest charlie-invok @id))))
+
+      (testing "unauthorized invocation user is denied fetching a request"
+        (let [func "jump"
+              exec-ftr (future (.execute bob-client func "some args"))
+              id (atom "")
+              expected "Only authorized users can fetchRequest"]
+          (Thread/sleep 1000)
+          (is (thrown-cause? AuthorizationException
+            (-> alice-invok (.fetchRequest func) .get_request_id)))
+          (with-timeout drpc-timeout-seconds TimeUnit/SECONDS
+            (while (empty? @id)
+              (reset! id
+                      (-> charlie-invok (.fetchRequest func) .get_request_id)))
+          (.result charlie-invok @id expected)
+          (is (= expected (.get exec-ftr drpc-timeout-seconds TimeUnit/SECONDS)))))))))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/backtype/storm/security/auth/nimbus_auth_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/security/auth/nimbus_auth_test.clj b/storm-core/test/clj/backtype/storm/security/auth/nimbus_auth_test.clj
new file mode 100644
index 0000000..bb70239
--- /dev/null
+++ b/storm-core/test/clj/backtype/storm/security/auth/nimbus_auth_test.clj
@@ -0,0 +1,181 @@
+;; 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.security.auth.nimbus-auth-test
+  (:use [clojure test])
+  (:require [backtype.storm [testing :as testing]])
+  (:require [backtype.storm.daemon [nimbus :as nimbus]])
+  (:require [backtype.storm [zookeeper :as zk]])
+  (:import [java.nio ByteBuffer])
+  (:import [backtype.storm Config])
+  (:import [backtype.storm.utils NimbusClient])
+  (:import [backtype.storm.generated NotAliveException])
+  (:import [backtype.storm.security.auth AuthUtils ThriftServer ThriftClient 
+                                         ReqContext ThriftConnectionType])
+  (:use [backtype.storm bootstrap cluster util])
+  (:use [backtype.storm.daemon common nimbus])
+  (:use [backtype.storm bootstrap])
+  (:import [backtype.storm.generated Nimbus Nimbus$Client 
+            AuthorizationException SubmitOptions TopologyInitialStatus KillOptions])
+  (:require [conjure.core])
+  (:use [conjure core])
+  )
+
+(bootstrap)
+
+(def nimbus-timeout (Integer. 30))
+
+(defn launch-test-cluster [nimbus-port login-cfg aznClass transportPluginClass] 
+  (let [conf {NIMBUS-AUTHORIZER aznClass 
+              NIMBUS-THRIFT-PORT nimbus-port
+              STORM-THRIFT-TRANSPORT-PLUGIN transportPluginClass }
+        conf (if login-cfg (merge conf {"java.security.auth.login.config" login-cfg}) conf)
+        cluster-map (testing/mk-local-storm-cluster :supervisors 0
+                                            :ports-per-supervisor 0
+                                            :daemon-conf conf)
+        nimbus-server (ThriftServer. (:daemon-conf cluster-map)
+                                     (Nimbus$Processor. (:nimbus cluster-map)) 
+                                     ThriftConnectionType/NIMBUS)]
+    (.addShutdownHook (Runtime/getRuntime) (Thread. (fn [] (.stop nimbus-server))))
+    (.start (Thread. #(.serve nimbus-server)))
+    (wait-for-condition #(.isServing nimbus-server))
+    [cluster-map nimbus-server]))
+
+(defmacro with-test-cluster [args & body]
+  `(let [[cluster-map#  nimbus-server#] (launch-test-cluster  ~@args)]
+      ~@body
+      (log-debug "Shutdown cluster from macro")
+      (testing/kill-local-storm-cluster cluster-map#)
+      (.stop nimbus-server#)))
+
+(deftest Simple-authentication-test 
+  (with-test-cluster [6627 nil nil "backtype.storm.security.auth.SimpleTransportPlugin"]
+    (let [storm-conf (merge (read-storm-config)
+                            {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin"
+                             STORM-NIMBUS-RETRY-TIMES 0})
+          client (NimbusClient. storm-conf "localhost" 6627 nimbus-timeout)
+          nimbus_client (.getClient client)]
+      (testing "(Positive authorization) Simple protocol w/o authentication/authorization enforcement"
+               (is (thrown-cause? NotAliveException
+                            (.activate nimbus_client "topo-name"))))
+      (.close client))))
+  
+(deftest test-noop-authorization-w-simple-transport 
+  (with-test-cluster [6628 nil 
+                "backtype.storm.security.auth.authorizer.NoopAuthorizer" 
+                "backtype.storm.security.auth.SimpleTransportPlugin"]
+    (let [storm-conf (merge (read-storm-config)
+                             {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin"
+                              STORM-NIMBUS-RETRY-TIMES 0})
+          client (NimbusClient. storm-conf "localhost" 6628 nimbus-timeout)
+          nimbus_client (.getClient client)]
+      (testing "(Positive authorization) Authorization plugin should accept client request"
+               (is (thrown-cause? NotAliveException
+                            (.activate nimbus_client "topo-name"))))
+      (.close client))))
+
+(deftest test-deny-authorization-w-simple-transport 
+  (with-test-cluster [6629 nil
+                "backtype.storm.security.auth.authorizer.DenyAuthorizer" 
+                "backtype.storm.security.auth.SimpleTransportPlugin"]
+    (let [storm-conf (merge (read-storm-config)
+                             {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin"
+                             Config/NIMBUS_HOST "localhost"
+                             Config/NIMBUS_THRIFT_PORT 6629
+                             STORM-NIMBUS-RETRY-TIMES 0})
+          client (NimbusClient/getConfiguredClient storm-conf)
+          nimbus_client (.getClient client)
+          topologyInitialStatus (TopologyInitialStatus/findByValue 2)
+          submitOptions (SubmitOptions. topologyInitialStatus)]
+      (is (thrown-cause? AuthorizationException (.submitTopology nimbus_client  "topo-name" nil nil nil))) 
+      (is (thrown-cause? AuthorizationException (.submitTopologyWithOpts nimbus_client  "topo-name" nil nil nil submitOptions)))
+      (is (thrown-cause? AuthorizationException (.beginFileUpload nimbus_client)))
+      (is (thrown-cause? AuthorizationException (.uploadChunk nimbus_client nil nil)))
+      (is (thrown-cause? AuthorizationException (.finishFileUpload nimbus_client nil)))
+      (is (thrown-cause? AuthorizationException (.beginFileDownload nimbus_client nil)))
+      (is (thrown-cause? AuthorizationException (.downloadChunk nimbus_client nil)))
+      (is (thrown-cause? AuthorizationException (.getNimbusConf nimbus_client)))
+      (is (thrown-cause? AuthorizationException (.getClusterInfo nimbus_client)))
+      (stubbing [nimbus/check-storm-active! nil
+                 nimbus/try-read-storm-conf-from-name {}]
+        (is (thrown-cause? AuthorizationException (.killTopology nimbus_client "topo-name")))
+        (is (thrown-cause? AuthorizationException (.killTopologyWithOpts nimbus_client "topo-name" (KillOptions.))))
+        (is (thrown-cause? AuthorizationException (.activate nimbus_client "topo-name")))
+        (is (thrown-cause? AuthorizationException (.deactivate nimbus_client "topo-name")))
+        (is (thrown-cause? AuthorizationException (.rebalance nimbus_client "topo-name" nil)))
+      )
+      (stubbing [nimbus/try-read-storm-conf {}]
+        (is (thrown-cause? AuthorizationException (.getTopologyConf nimbus_client "topo-ID")))
+        (is (thrown-cause? AuthorizationException (.getTopology nimbus_client "topo-ID")))
+        (is (thrown-cause? AuthorizationException (.getUserTopology nimbus_client "topo-ID")))
+        (is (thrown-cause? AuthorizationException (.getTopologyInfo nimbus_client "topo-ID"))))
+      (.close client))))
+
+(deftest test-noop-authorization-w-sasl-digest 
+  (with-test-cluster [6630
+                "test/clj/backtype/storm/security/auth/jaas_digest.conf" 
+                "backtype.storm.security.auth.authorizer.NoopAuthorizer" 
+                "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"]
+    (let [storm-conf (merge (read-storm-config)
+                            {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"
+                             "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest.conf"
+                             Config/NIMBUS_HOST "localhost"
+                             Config/NIMBUS_THRIFT_PORT 6630
+                             STORM-NIMBUS-RETRY-TIMES 0})
+          client (NimbusClient/getConfiguredClient storm-conf)
+          nimbus_client (.getClient client)]
+      (testing "(Positive authorization) Authorization plugin should accept client request"
+               (is (thrown-cause? NotAliveException
+                            (.activate nimbus_client "topo-name"))))
+      (.close client))))
+
+(deftest test-deny-authorization-w-sasl-digest 
+  (with-test-cluster [6631
+                "test/clj/backtype/storm/security/auth/jaas_digest.conf" 
+                "backtype.storm.security.auth.authorizer.DenyAuthorizer" 
+                "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"]
+    (let [storm-conf (merge (read-storm-config)
+                            {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"
+                             "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest.conf"
+                             Config/NIMBUS_HOST "localhost"
+                             Config/NIMBUS_THRIFT_PORT 6631
+                             STORM-NIMBUS-RETRY-TIMES 0})
+          client (NimbusClient/getConfiguredClient storm-conf)
+          nimbus_client (.getClient client)
+          topologyInitialStatus (TopologyInitialStatus/findByValue 2)
+          submitOptions (SubmitOptions. topologyInitialStatus)]
+      (is (thrown-cause? AuthorizationException (.submitTopology nimbus_client  "topo-name" nil nil nil))) 
+      (is (thrown-cause? AuthorizationException (.submitTopologyWithOpts nimbus_client  "topo-name" nil nil nil submitOptions)))
+      (is (thrown-cause? AuthorizationException (.beginFileUpload nimbus_client)))
+      (is (thrown-cause? AuthorizationException (.uploadChunk nimbus_client nil nil)))
+      (is (thrown-cause? AuthorizationException (.finishFileUpload nimbus_client nil)))
+      (is (thrown-cause? AuthorizationException (.beginFileDownload nimbus_client nil)))
+      (is (thrown-cause? AuthorizationException (.downloadChunk nimbus_client nil)))
+      (is (thrown-cause? AuthorizationException (.getNimbusConf nimbus_client)))
+      (is (thrown-cause? AuthorizationException (.getClusterInfo nimbus_client)))
+      (stubbing [nimbus/check-storm-active! nil
+                 nimbus/try-read-storm-conf-from-name {}]
+        (is (thrown-cause? AuthorizationException (.killTopology nimbus_client "topo-name")))
+        (is (thrown-cause? AuthorizationException (.killTopologyWithOpts nimbus_client "topo-name" (KillOptions.))))
+        (is (thrown-cause? AuthorizationException (.activate nimbus_client "topo-name")))
+        (is (thrown-cause? AuthorizationException (.deactivate nimbus_client "topo-name")))
+        (is (thrown-cause? AuthorizationException (.rebalance nimbus_client "topo-name" nil))))
+      (stubbing [nimbus/try-read-storm-conf {}]
+        (is (thrown-cause? AuthorizationException (.getTopologyConf nimbus_client "topo-ID")))
+        (is (thrown-cause? AuthorizationException (.getTopology nimbus_client "topo-ID")))
+        (is (thrown-cause? AuthorizationException (.getUserTopology nimbus_client "topo-ID")))
+        (is (thrown-cause? AuthorizationException (.getTopologyInfo nimbus_client "topo-ID"))))
+      (.close client))))
+

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/test/clj/backtype/storm/submitter_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/submitter_test.clj b/storm-core/test/clj/backtype/storm/submitter_test.clj
new file mode 100644
index 0000000..6f88297
--- /dev/null
+++ b/storm-core/test/clj/backtype/storm/submitter_test.clj
@@ -0,0 +1,75 @@
+;; 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.submitter-test
+  (:use [clojure test])
+  (:use [backtype.storm config testing])
+  (:import [backtype.storm StormSubmitter])
+  )
+
+(deftest test-md5-digest-secret-generation
+  (testing "No payload or scheme are generated when already present"
+    (let [conf {STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD "foobar:12345"
+                STORM-ZOOKEEPER-AUTH-SCHEME "anything"}
+          result (StormSubmitter/prepareZookeeperAuthentication conf)
+          actual-payload (.get result STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD)
+          actual-scheme (.get result STORM-ZOOKEEPER-TOPOLOGY-AUTH-SCHEME)]
+      (is (nil? actual-payload))
+      (is (= "digest" actual-scheme))))
+
+  (testing "Scheme is set to digest if not already."
+    (let [conf {STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD "foobar:12345"}
+          result (StormSubmitter/prepareZookeeperAuthentication conf)
+          actual-payload (.get result STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD)
+          actual-scheme (.get result STORM-ZOOKEEPER-TOPOLOGY-AUTH-SCHEME)]
+      (is (nil? actual-payload))
+      (is (= "digest" actual-scheme))))
+
+  (testing "A payload is generated when no payload is present."
+    (let [conf {STORM-ZOOKEEPER-AUTH-SCHEME "anything"}
+          result (StormSubmitter/prepareZookeeperAuthentication conf)
+          actual-payload (.get result STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD)
+          actual-scheme (.get result STORM-ZOOKEEPER-TOPOLOGY-AUTH-SCHEME)]
+      (is (not (clojure.string/blank? actual-payload)))
+      (is (= "digest" actual-scheme))))
+
+  (testing "A payload is generated when payload is not correctly formatted."
+    (let [bogus-payload "not-a-valid-payload"
+          conf {STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD bogus-payload
+                STORM-ZOOKEEPER-AUTH-SCHEME "anything"}
+          result (StormSubmitter/prepareZookeeperAuthentication conf)
+          actual-payload (.get result STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD)
+          actual-scheme (.get result STORM-ZOOKEEPER-TOPOLOGY-AUTH-SCHEME)]
+      (is (not (StormSubmitter/validateZKDigestPayload bogus-payload))) ; Is this test correct?
+      (is (not (clojure.string/blank? actual-payload)))
+      (is (= "digest" actual-scheme))))
+
+  (testing "A payload is generated when payload is null."
+    (let [conf {STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD nil
+                STORM-ZOOKEEPER-AUTH-SCHEME "anything"}
+          result (StormSubmitter/prepareZookeeperAuthentication conf)
+          actual-payload (.get result STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD)
+          actual-scheme (.get result STORM-ZOOKEEPER-TOPOLOGY-AUTH-SCHEME)]
+      (is (not (clojure.string/blank? actual-payload)))
+      (is (= "digest" actual-scheme))))
+
+  (testing "A payload is generated when payload is blank."
+    (let [conf {STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD ""
+                STORM-ZOOKEEPER-AUTH-SCHEME "anything"}
+          result (StormSubmitter/prepareZookeeperAuthentication conf)
+          actual-payload (.get result STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD)
+          actual-scheme (.get result STORM-ZOOKEEPER-TOPOLOGY-AUTH-SCHEME)]
+      (is (not (clojure.string/blank? actual-payload)))
+      (is (= "digest" actual-scheme)))))


[04/14] STORM-216: Added Authentication and Authorization.

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/py/storm/Nimbus.py
----------------------------------------------------------------------
diff --git a/storm-core/src/py/storm/Nimbus.py b/storm-core/src/py/storm/Nimbus.py
index cd535be..e1ee510 100644
--- a/storm-core/src/py/storm/Nimbus.py
+++ b/storm-core/src/py/storm/Nimbus.py
@@ -74,6 +74,14 @@ class Iface:
     """
     pass
 
+  def uploadNewCredentials(self, name, creds):
+    """
+    Parameters:
+     - name
+     - creds
+    """
+    pass
+
   def beginFileUpload(self, ):
     pass
 
@@ -184,6 +192,8 @@ class Client(Iface):
       raise result.e
     if result.ite is not None:
       raise result.ite
+    if result.aze is not None:
+      raise result.aze
     return
 
   def submitTopologyWithOpts(self, name, uploadedJarLocation, jsonConf, topology, options):
@@ -224,6 +234,8 @@ class Client(Iface):
       raise result.e
     if result.ite is not None:
       raise result.ite
+    if result.aze is not None:
+      raise result.aze
     return
 
   def killTopology(self, name):
@@ -254,6 +266,8 @@ class Client(Iface):
     self._iprot.readMessageEnd()
     if result.e is not None:
       raise result.e
+    if result.aze is not None:
+      raise result.aze
     return
 
   def killTopologyWithOpts(self, name, options):
@@ -286,6 +300,8 @@ class Client(Iface):
     self._iprot.readMessageEnd()
     if result.e is not None:
       raise result.e
+    if result.aze is not None:
+      raise result.aze
     return
 
   def activate(self, name):
@@ -316,6 +332,8 @@ class Client(Iface):
     self._iprot.readMessageEnd()
     if result.e is not None:
       raise result.e
+    if result.aze is not None:
+      raise result.aze
     return
 
   def deactivate(self, name):
@@ -346,6 +364,8 @@ class Client(Iface):
     self._iprot.readMessageEnd()
     if result.e is not None:
       raise result.e
+    if result.aze is not None:
+      raise result.aze
     return
 
   def rebalance(self, name, options):
@@ -380,6 +400,44 @@ class Client(Iface):
       raise result.e
     if result.ite is not None:
       raise result.ite
+    if result.aze is not None:
+      raise result.aze
+    return
+
+  def uploadNewCredentials(self, name, creds):
+    """
+    Parameters:
+     - name
+     - creds
+    """
+    self.send_uploadNewCredentials(name, creds)
+    self.recv_uploadNewCredentials()
+
+  def send_uploadNewCredentials(self, name, creds):
+    self._oprot.writeMessageBegin('uploadNewCredentials', TMessageType.CALL, self._seqid)
+    args = uploadNewCredentials_args()
+    args.name = name
+    args.creds = creds
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_uploadNewCredentials(self, ):
+    (fname, mtype, rseqid) = self._iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(self._iprot)
+      self._iprot.readMessageEnd()
+      raise x
+    result = uploadNewCredentials_result()
+    result.read(self._iprot)
+    self._iprot.readMessageEnd()
+    if result.e is not None:
+      raise result.e
+    if result.ite is not None:
+      raise result.ite
+    if result.aze is not None:
+      raise result.aze
     return
 
   def beginFileUpload(self, ):
@@ -405,6 +463,8 @@ class Client(Iface):
     self._iprot.readMessageEnd()
     if result.success is not None:
       return result.success
+    if result.aze is not None:
+      raise result.aze
     raise TApplicationException(TApplicationException.MISSING_RESULT, "beginFileUpload failed: unknown result");
 
   def uploadChunk(self, location, chunk):
@@ -435,6 +495,8 @@ class Client(Iface):
     result = uploadChunk_result()
     result.read(self._iprot)
     self._iprot.readMessageEnd()
+    if result.aze is not None:
+      raise result.aze
     return
 
   def finishFileUpload(self, location):
@@ -463,6 +525,8 @@ class Client(Iface):
     result = finishFileUpload_result()
     result.read(self._iprot)
     self._iprot.readMessageEnd()
+    if result.aze is not None:
+      raise result.aze
     return
 
   def beginFileDownload(self, file):
@@ -493,6 +557,8 @@ class Client(Iface):
     self._iprot.readMessageEnd()
     if result.success is not None:
       return result.success
+    if result.aze is not None:
+      raise result.aze
     raise TApplicationException(TApplicationException.MISSING_RESULT, "beginFileDownload failed: unknown result");
 
   def downloadChunk(self, id):
@@ -523,6 +589,8 @@ class Client(Iface):
     self._iprot.readMessageEnd()
     if result.success is not None:
       return result.success
+    if result.aze is not None:
+      raise result.aze
     raise TApplicationException(TApplicationException.MISSING_RESULT, "downloadChunk failed: unknown result");
 
   def getNimbusConf(self, ):
@@ -548,6 +616,8 @@ class Client(Iface):
     self._iprot.readMessageEnd()
     if result.success is not None:
       return result.success
+    if result.aze is not None:
+      raise result.aze
     raise TApplicationException(TApplicationException.MISSING_RESULT, "getNimbusConf failed: unknown result");
 
   def getClusterInfo(self, ):
@@ -573,6 +643,8 @@ class Client(Iface):
     self._iprot.readMessageEnd()
     if result.success is not None:
       return result.success
+    if result.aze is not None:
+      raise result.aze
     raise TApplicationException(TApplicationException.MISSING_RESULT, "getClusterInfo failed: unknown result");
 
   def getTopologyInfo(self, id):
@@ -605,6 +677,8 @@ class Client(Iface):
       return result.success
     if result.e is not None:
       raise result.e
+    if result.aze is not None:
+      raise result.aze
     raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyInfo failed: unknown result");
 
   def getTopologyConf(self, id):
@@ -637,6 +711,8 @@ class Client(Iface):
       return result.success
     if result.e is not None:
       raise result.e
+    if result.aze is not None:
+      raise result.aze
     raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyConf failed: unknown result");
 
   def getTopology(self, id):
@@ -669,6 +745,8 @@ class Client(Iface):
       return result.success
     if result.e is not None:
       raise result.e
+    if result.aze is not None:
+      raise result.aze
     raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopology failed: unknown result");
 
   def getUserTopology(self, id):
@@ -701,6 +779,8 @@ class Client(Iface):
       return result.success
     if result.e is not None:
       raise result.e
+    if result.aze is not None:
+      raise result.aze
     raise TApplicationException(TApplicationException.MISSING_RESULT, "getUserTopology failed: unknown result");
 
 
@@ -715,6 +795,7 @@ class Processor(Iface, TProcessor):
     self._processMap["activate"] = Processor.process_activate
     self._processMap["deactivate"] = Processor.process_deactivate
     self._processMap["rebalance"] = Processor.process_rebalance
+    self._processMap["uploadNewCredentials"] = Processor.process_uploadNewCredentials
     self._processMap["beginFileUpload"] = Processor.process_beginFileUpload
     self._processMap["uploadChunk"] = Processor.process_uploadChunk
     self._processMap["finishFileUpload"] = Processor.process_finishFileUpload
@@ -753,6 +834,8 @@ class Processor(Iface, TProcessor):
       result.e = e
     except InvalidTopologyException, ite:
       result.ite = ite
+    except AuthorizationException, aze:
+      result.aze = aze
     oprot.writeMessageBegin("submitTopology", TMessageType.REPLY, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
@@ -769,6 +852,8 @@ class Processor(Iface, TProcessor):
       result.e = e
     except InvalidTopologyException, ite:
       result.ite = ite
+    except AuthorizationException, aze:
+      result.aze = aze
     oprot.writeMessageBegin("submitTopologyWithOpts", TMessageType.REPLY, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
@@ -783,6 +868,8 @@ class Processor(Iface, TProcessor):
       self._handler.killTopology(args.name)
     except NotAliveException, e:
       result.e = e
+    except AuthorizationException, aze:
+      result.aze = aze
     oprot.writeMessageBegin("killTopology", TMessageType.REPLY, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
@@ -797,6 +884,8 @@ class Processor(Iface, TProcessor):
       self._handler.killTopologyWithOpts(args.name, args.options)
     except NotAliveException, e:
       result.e = e
+    except AuthorizationException, aze:
+      result.aze = aze
     oprot.writeMessageBegin("killTopologyWithOpts", TMessageType.REPLY, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
@@ -811,6 +900,8 @@ class Processor(Iface, TProcessor):
       self._handler.activate(args.name)
     except NotAliveException, e:
       result.e = e
+    except AuthorizationException, aze:
+      result.aze = aze
     oprot.writeMessageBegin("activate", TMessageType.REPLY, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
@@ -825,6 +916,8 @@ class Processor(Iface, TProcessor):
       self._handler.deactivate(args.name)
     except NotAliveException, e:
       result.e = e
+    except AuthorizationException, aze:
+      result.aze = aze
     oprot.writeMessageBegin("deactivate", TMessageType.REPLY, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
@@ -841,17 +934,40 @@ class Processor(Iface, TProcessor):
       result.e = e
     except InvalidTopologyException, ite:
       result.ite = ite
+    except AuthorizationException, aze:
+      result.aze = aze
     oprot.writeMessageBegin("rebalance", TMessageType.REPLY, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_uploadNewCredentials(self, seqid, iprot, oprot):
+    args = uploadNewCredentials_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = uploadNewCredentials_result()
+    try:
+      self._handler.uploadNewCredentials(args.name, args.creds)
+    except NotAliveException, e:
+      result.e = e
+    except InvalidTopologyException, ite:
+      result.ite = ite
+    except AuthorizationException, aze:
+      result.aze = aze
+    oprot.writeMessageBegin("uploadNewCredentials", TMessageType.REPLY, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
   def process_beginFileUpload(self, seqid, iprot, oprot):
     args = beginFileUpload_args()
     args.read(iprot)
     iprot.readMessageEnd()
     result = beginFileUpload_result()
-    result.success = self._handler.beginFileUpload()
+    try:
+      result.success = self._handler.beginFileUpload()
+    except AuthorizationException, aze:
+      result.aze = aze
     oprot.writeMessageBegin("beginFileUpload", TMessageType.REPLY, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
@@ -862,7 +978,10 @@ class Processor(Iface, TProcessor):
     args.read(iprot)
     iprot.readMessageEnd()
     result = uploadChunk_result()
-    self._handler.uploadChunk(args.location, args.chunk)
+    try:
+      self._handler.uploadChunk(args.location, args.chunk)
+    except AuthorizationException, aze:
+      result.aze = aze
     oprot.writeMessageBegin("uploadChunk", TMessageType.REPLY, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
@@ -873,7 +992,10 @@ class Processor(Iface, TProcessor):
     args.read(iprot)
     iprot.readMessageEnd()
     result = finishFileUpload_result()
-    self._handler.finishFileUpload(args.location)
+    try:
+      self._handler.finishFileUpload(args.location)
+    except AuthorizationException, aze:
+      result.aze = aze
     oprot.writeMessageBegin("finishFileUpload", TMessageType.REPLY, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
@@ -884,7 +1006,10 @@ class Processor(Iface, TProcessor):
     args.read(iprot)
     iprot.readMessageEnd()
     result = beginFileDownload_result()
-    result.success = self._handler.beginFileDownload(args.file)
+    try:
+      result.success = self._handler.beginFileDownload(args.file)
+    except AuthorizationException, aze:
+      result.aze = aze
     oprot.writeMessageBegin("beginFileDownload", TMessageType.REPLY, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
@@ -895,7 +1020,10 @@ class Processor(Iface, TProcessor):
     args.read(iprot)
     iprot.readMessageEnd()
     result = downloadChunk_result()
-    result.success = self._handler.downloadChunk(args.id)
+    try:
+      result.success = self._handler.downloadChunk(args.id)
+    except AuthorizationException, aze:
+      result.aze = aze
     oprot.writeMessageBegin("downloadChunk", TMessageType.REPLY, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
@@ -906,7 +1034,10 @@ class Processor(Iface, TProcessor):
     args.read(iprot)
     iprot.readMessageEnd()
     result = getNimbusConf_result()
-    result.success = self._handler.getNimbusConf()
+    try:
+      result.success = self._handler.getNimbusConf()
+    except AuthorizationException, aze:
+      result.aze = aze
     oprot.writeMessageBegin("getNimbusConf", TMessageType.REPLY, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
@@ -917,7 +1048,10 @@ class Processor(Iface, TProcessor):
     args.read(iprot)
     iprot.readMessageEnd()
     result = getClusterInfo_result()
-    result.success = self._handler.getClusterInfo()
+    try:
+      result.success = self._handler.getClusterInfo()
+    except AuthorizationException, aze:
+      result.aze = aze
     oprot.writeMessageBegin("getClusterInfo", TMessageType.REPLY, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
@@ -932,6 +1066,8 @@ class Processor(Iface, TProcessor):
       result.success = self._handler.getTopologyInfo(args.id)
     except NotAliveException, e:
       result.e = e
+    except AuthorizationException, aze:
+      result.aze = aze
     oprot.writeMessageBegin("getTopologyInfo", TMessageType.REPLY, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
@@ -946,6 +1082,8 @@ class Processor(Iface, TProcessor):
       result.success = self._handler.getTopologyConf(args.id)
     except NotAliveException, e:
       result.e = e
+    except AuthorizationException, aze:
+      result.aze = aze
     oprot.writeMessageBegin("getTopologyConf", TMessageType.REPLY, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
@@ -960,6 +1098,8 @@ class Processor(Iface, TProcessor):
       result.success = self._handler.getTopology(args.id)
     except NotAliveException, e:
       result.e = e
+    except AuthorizationException, aze:
+      result.aze = aze
     oprot.writeMessageBegin("getTopology", TMessageType.REPLY, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
@@ -974,6 +1114,8 @@ class Processor(Iface, TProcessor):
       result.success = self._handler.getUserTopology(args.id)
     except NotAliveException, e:
       result.e = e
+    except AuthorizationException, aze:
+      result.aze = aze
     oprot.writeMessageBegin("getUserTopology", TMessageType.REPLY, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
@@ -1087,20 +1229,23 @@ class submitTopology_result:
   Attributes:
    - e
    - ite
+   - aze
   """
 
   thrift_spec = (
     None, # 0
     (1, TType.STRUCT, 'e', (AlreadyAliveException, AlreadyAliveException.thrift_spec), None, ), # 1
     (2, TType.STRUCT, 'ite', (InvalidTopologyException, InvalidTopologyException.thrift_spec), None, ), # 2
+    (3, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 3
   )
 
   def __hash__(self):
-    return 0 + hash(self.e) + hash(self.ite)
+    return 0 + hash(self.e) + hash(self.ite) + hash(self.aze)
 
-  def __init__(self, e=None, ite=None,):
+  def __init__(self, e=None, ite=None, aze=None,):
     self.e = e
     self.ite = ite
+    self.aze = aze
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -1123,6 +1268,12 @@ class submitTopology_result:
           self.ite.read(iprot)
         else:
           iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -1141,6 +1292,10 @@ class submitTopology_result:
       oprot.writeFieldBegin('ite', TType.STRUCT, 2)
       self.ite.write(oprot)
       oprot.writeFieldEnd()
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 3)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -1277,20 +1432,23 @@ class submitTopologyWithOpts_result:
   Attributes:
    - e
    - ite
+   - aze
   """
 
   thrift_spec = (
     None, # 0
     (1, TType.STRUCT, 'e', (AlreadyAliveException, AlreadyAliveException.thrift_spec), None, ), # 1
     (2, TType.STRUCT, 'ite', (InvalidTopologyException, InvalidTopologyException.thrift_spec), None, ), # 2
+    (3, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 3
   )
 
   def __hash__(self):
-    return 0 + hash(self.e) + hash(self.ite)
+    return 0 + hash(self.e) + hash(self.ite) + hash(self.aze)
 
-  def __init__(self, e=None, ite=None,):
+  def __init__(self, e=None, ite=None, aze=None,):
     self.e = e
     self.ite = ite
+    self.aze = aze
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -1313,6 +1471,12 @@ class submitTopologyWithOpts_result:
           self.ite.read(iprot)
         else:
           iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -1331,6 +1495,10 @@ class submitTopologyWithOpts_result:
       oprot.writeFieldBegin('ite', TType.STRUCT, 2)
       self.ite.write(oprot)
       oprot.writeFieldEnd()
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 3)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -1416,18 +1584,21 @@ class killTopology_result:
   """
   Attributes:
    - e
+   - aze
   """
 
   thrift_spec = (
     None, # 0
     (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 2
   )
 
   def __hash__(self):
-    return 0 + hash(self.e)
+    return 0 + hash(self.e) + hash(self.aze)
 
-  def __init__(self, e=None,):
+  def __init__(self, e=None, aze=None,):
     self.e = e
+    self.aze = aze
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -1444,6 +1615,12 @@ class killTopology_result:
           self.e.read(iprot)
         else:
           iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -1458,6 +1635,10 @@ class killTopology_result:
       oprot.writeFieldBegin('e', TType.STRUCT, 1)
       self.e.write(oprot)
       oprot.writeFieldEnd()
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 2)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -1556,18 +1737,21 @@ class killTopologyWithOpts_result:
   """
   Attributes:
    - e
+   - aze
   """
 
   thrift_spec = (
     None, # 0
     (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 2
   )
 
   def __hash__(self):
-    return 0 + hash(self.e)
+    return 0 + hash(self.e) + hash(self.aze)
 
-  def __init__(self, e=None,):
+  def __init__(self, e=None, aze=None,):
     self.e = e
+    self.aze = aze
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -1584,6 +1768,12 @@ class killTopologyWithOpts_result:
           self.e.read(iprot)
         else:
           iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -1598,6 +1788,10 @@ class killTopologyWithOpts_result:
       oprot.writeFieldBegin('e', TType.STRUCT, 1)
       self.e.write(oprot)
       oprot.writeFieldEnd()
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 2)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -1683,18 +1877,21 @@ class activate_result:
   """
   Attributes:
    - e
+   - aze
   """
 
   thrift_spec = (
     None, # 0
     (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 2
   )
 
   def __hash__(self):
-    return 0 + hash(self.e)
+    return 0 + hash(self.e) + hash(self.aze)
 
-  def __init__(self, e=None,):
+  def __init__(self, e=None, aze=None,):
     self.e = e
+    self.aze = aze
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -1711,6 +1908,12 @@ class activate_result:
           self.e.read(iprot)
         else:
           iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -1725,6 +1928,10 @@ class activate_result:
       oprot.writeFieldBegin('e', TType.STRUCT, 1)
       self.e.write(oprot)
       oprot.writeFieldEnd()
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 2)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -1810,18 +2017,21 @@ class deactivate_result:
   """
   Attributes:
    - e
+   - aze
   """
 
   thrift_spec = (
     None, # 0
     (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 2
   )
 
   def __hash__(self):
-    return 0 + hash(self.e)
+    return 0 + hash(self.e) + hash(self.aze)
 
-  def __init__(self, e=None,):
+  def __init__(self, e=None, aze=None,):
     self.e = e
+    self.aze = aze
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -1838,6 +2048,12 @@ class deactivate_result:
           self.e.read(iprot)
         else:
           iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -1852,6 +2068,10 @@ class deactivate_result:
       oprot.writeFieldBegin('e', TType.STRUCT, 1)
       self.e.write(oprot)
       oprot.writeFieldEnd()
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 2)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -1951,20 +2171,23 @@ class rebalance_result:
   Attributes:
    - e
    - ite
+   - aze
   """
 
   thrift_spec = (
     None, # 0
     (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1
     (2, TType.STRUCT, 'ite', (InvalidTopologyException, InvalidTopologyException.thrift_spec), None, ), # 2
+    (3, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 3
   )
 
   def __hash__(self):
-    return 0 + hash(self.e) + hash(self.ite)
+    return 0 + hash(self.e) + hash(self.ite) + hash(self.aze)
 
-  def __init__(self, e=None, ite=None,):
+  def __init__(self, e=None, ite=None, aze=None,):
     self.e = e
     self.ite = ite
+    self.aze = aze
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -1987,6 +2210,12 @@ class rebalance_result:
           self.ite.read(iprot)
         else:
           iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -2005,6 +2234,176 @@ class rebalance_result:
       oprot.writeFieldBegin('ite', TType.STRUCT, 2)
       self.ite.write(oprot)
       oprot.writeFieldEnd()
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 3)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class uploadNewCredentials_args:
+  """
+  Attributes:
+   - name
+   - creds
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'name', None, None, ), # 1
+    (2, TType.STRUCT, 'creds', (Credentials, Credentials.thrift_spec), None, ), # 2
+  )
+
+  def __hash__(self):
+    return 0 + hash(self.name) + hash(self.creds)
+
+  def __init__(self, name=None, creds=None,):
+    self.name = name
+    self.creds = creds
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.name = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.creds = Credentials()
+          self.creds.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('uploadNewCredentials_args')
+    if self.name is not None:
+      oprot.writeFieldBegin('name', TType.STRING, 1)
+      oprot.writeString(self.name.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.creds is not None:
+      oprot.writeFieldBegin('creds', TType.STRUCT, 2)
+      self.creds.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class uploadNewCredentials_result:
+  """
+  Attributes:
+   - e
+   - ite
+   - aze
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'ite', (InvalidTopologyException, InvalidTopologyException.thrift_spec), None, ), # 2
+    (3, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 3
+  )
+
+  def __hash__(self):
+    return 0 + hash(self.e) + hash(self.ite) + hash(self.aze)
+
+  def __init__(self, e=None, ite=None, aze=None,):
+    self.e = e
+    self.ite = ite
+    self.aze = aze
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.e = NotAliveException()
+          self.e.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.ite = InvalidTopologyException()
+          self.ite.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('uploadNewCredentials_result')
+    if self.e is not None:
+      oprot.writeFieldBegin('e', TType.STRUCT, 1)
+      self.e.write(oprot)
+      oprot.writeFieldEnd()
+    if self.ite is not None:
+      oprot.writeFieldBegin('ite', TType.STRUCT, 2)
+      self.ite.write(oprot)
+      oprot.writeFieldEnd()
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 3)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -2072,17 +2471,20 @@ class beginFileUpload_result:
   """
   Attributes:
    - success
+   - aze
   """
 
   thrift_spec = (
     (0, TType.STRING, 'success', None, None, ), # 0
+    (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1
   )
 
   def __hash__(self):
-    return 0 + hash(self.success)
+    return 0 + hash(self.success) + hash(self.aze)
 
-  def __init__(self, success=None,):
+  def __init__(self, success=None, aze=None,):
     self.success = success
+    self.aze = aze
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -2098,6 +2500,12 @@ class beginFileUpload_result:
           self.success = iprot.readString().decode('utf-8')
         else:
           iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -2112,6 +2520,10 @@ class beginFileUpload_result:
       oprot.writeFieldBegin('success', TType.STRING, 0)
       oprot.writeString(self.success.encode('utf-8'))
       oprot.writeFieldEnd()
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 1)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -2206,12 +2618,21 @@ class uploadChunk_args:
     return not (self == other)
 
 class uploadChunk_result:
+  """
+  Attributes:
+   - aze
+  """
 
   thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1
   )
 
   def __hash__(self):
-    return 0
+    return 0 + hash(self.aze)
+
+  def __init__(self, aze=None,):
+    self.aze = aze
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -2222,6 +2643,12 @@ class uploadChunk_result:
       (fname, ftype, fid) = iprot.readFieldBegin()
       if ftype == TType.STOP:
         break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -2232,6 +2659,10 @@ class uploadChunk_result:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
     oprot.writeStructBegin('uploadChunk_result')
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 1)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -2314,12 +2745,21 @@ class finishFileUpload_args:
     return not (self == other)
 
 class finishFileUpload_result:
+  """
+  Attributes:
+   - aze
+  """
 
   thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1
   )
 
   def __hash__(self):
-    return 0
+    return 0 + hash(self.aze)
+
+  def __init__(self, aze=None,):
+    self.aze = aze
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -2330,6 +2770,12 @@ class finishFileUpload_result:
       (fname, ftype, fid) = iprot.readFieldBegin()
       if ftype == TType.STOP:
         break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -2340,6 +2786,10 @@ class finishFileUpload_result:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
     oprot.writeStructBegin('finishFileUpload_result')
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 1)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -2425,17 +2875,20 @@ class beginFileDownload_result:
   """
   Attributes:
    - success
+   - aze
   """
 
   thrift_spec = (
     (0, TType.STRING, 'success', None, None, ), # 0
+    (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1
   )
 
   def __hash__(self):
-    return 0 + hash(self.success)
+    return 0 + hash(self.success) + hash(self.aze)
 
-  def __init__(self, success=None,):
+  def __init__(self, success=None, aze=None,):
     self.success = success
+    self.aze = aze
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -2451,6 +2904,12 @@ class beginFileDownload_result:
           self.success = iprot.readString().decode('utf-8')
         else:
           iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -2465,6 +2924,10 @@ class beginFileDownload_result:
       oprot.writeFieldBegin('success', TType.STRING, 0)
       oprot.writeString(self.success.encode('utf-8'))
       oprot.writeFieldEnd()
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 1)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -2550,17 +3013,20 @@ class downloadChunk_result:
   """
   Attributes:
    - success
+   - aze
   """
 
   thrift_spec = (
     (0, TType.STRING, 'success', None, None, ), # 0
+    (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1
   )
 
   def __hash__(self):
-    return 0 + hash(self.success)
+    return 0 + hash(self.success) + hash(self.aze)
 
-  def __init__(self, success=None,):
+  def __init__(self, success=None, aze=None,):
     self.success = success
+    self.aze = aze
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -2576,6 +3042,12 @@ class downloadChunk_result:
           self.success = iprot.readString();
         else:
           iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -2590,6 +3062,10 @@ class downloadChunk_result:
       oprot.writeFieldBegin('success', TType.STRING, 0)
       oprot.writeString(self.success)
       oprot.writeFieldEnd()
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 1)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -2657,17 +3133,20 @@ class getNimbusConf_result:
   """
   Attributes:
    - success
+   - aze
   """
 
   thrift_spec = (
     (0, TType.STRING, 'success', None, None, ), # 0
+    (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1
   )
 
   def __hash__(self):
-    return 0 + hash(self.success)
+    return 0 + hash(self.success) + hash(self.aze)
 
-  def __init__(self, success=None,):
+  def __init__(self, success=None, aze=None,):
     self.success = success
+    self.aze = aze
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -2683,6 +3162,12 @@ class getNimbusConf_result:
           self.success = iprot.readString().decode('utf-8')
         else:
           iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -2697,6 +3182,10 @@ class getNimbusConf_result:
       oprot.writeFieldBegin('success', TType.STRING, 0)
       oprot.writeString(self.success.encode('utf-8'))
       oprot.writeFieldEnd()
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 1)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -2764,17 +3253,20 @@ class getClusterInfo_result:
   """
   Attributes:
    - success
+   - aze
   """
 
   thrift_spec = (
     (0, TType.STRUCT, 'success', (ClusterSummary, ClusterSummary.thrift_spec), None, ), # 0
+    (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1
   )
 
   def __hash__(self):
-    return 0 + hash(self.success)
+    return 0 + hash(self.success) + hash(self.aze)
 
-  def __init__(self, success=None,):
+  def __init__(self, success=None, aze=None,):
     self.success = success
+    self.aze = aze
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -2791,6 +3283,12 @@ class getClusterInfo_result:
           self.success.read(iprot)
         else:
           iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -2805,6 +3303,10 @@ class getClusterInfo_result:
       oprot.writeFieldBegin('success', TType.STRUCT, 0)
       self.success.write(oprot)
       oprot.writeFieldEnd()
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 1)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -2891,19 +3393,22 @@ class getTopologyInfo_result:
   Attributes:
    - success
    - e
+   - aze
   """
 
   thrift_spec = (
     (0, TType.STRUCT, 'success', (TopologyInfo, TopologyInfo.thrift_spec), None, ), # 0
     (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 2
   )
 
   def __hash__(self):
-    return 0 + hash(self.success) + hash(self.e)
+    return 0 + hash(self.success) + hash(self.e) + hash(self.aze)
 
-  def __init__(self, success=None, e=None,):
+  def __init__(self, success=None, e=None, aze=None,):
     self.success = success
     self.e = e
+    self.aze = aze
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -2926,6 +3431,12 @@ class getTopologyInfo_result:
           self.e.read(iprot)
         else:
           iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -2944,6 +3455,10 @@ class getTopologyInfo_result:
       oprot.writeFieldBegin('e', TType.STRUCT, 1)
       self.e.write(oprot)
       oprot.writeFieldEnd()
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 2)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -3030,19 +3545,22 @@ class getTopologyConf_result:
   Attributes:
    - success
    - e
+   - aze
   """
 
   thrift_spec = (
     (0, TType.STRING, 'success', None, None, ), # 0
     (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 2
   )
 
   def __hash__(self):
-    return 0 + hash(self.success) + hash(self.e)
+    return 0 + hash(self.success) + hash(self.e) + hash(self.aze)
 
-  def __init__(self, success=None, e=None,):
+  def __init__(self, success=None, e=None, aze=None,):
     self.success = success
     self.e = e
+    self.aze = aze
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -3064,6 +3582,12 @@ class getTopologyConf_result:
           self.e.read(iprot)
         else:
           iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -3082,6 +3606,10 @@ class getTopologyConf_result:
       oprot.writeFieldBegin('e', TType.STRUCT, 1)
       self.e.write(oprot)
       oprot.writeFieldEnd()
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 2)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -3168,19 +3696,22 @@ class getTopology_result:
   Attributes:
    - success
    - e
+   - aze
   """
 
   thrift_spec = (
     (0, TType.STRUCT, 'success', (StormTopology, StormTopology.thrift_spec), None, ), # 0
     (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 2
   )
 
   def __hash__(self):
-    return 0 + hash(self.success) + hash(self.e)
+    return 0 + hash(self.success) + hash(self.e) + hash(self.aze)
 
-  def __init__(self, success=None, e=None,):
+  def __init__(self, success=None, e=None, aze=None,):
     self.success = success
     self.e = e
+    self.aze = aze
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -3203,6 +3734,12 @@ class getTopology_result:
           self.e.read(iprot)
         else:
           iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -3221,6 +3758,10 @@ class getTopology_result:
       oprot.writeFieldBegin('e', TType.STRUCT, 1)
       self.e.write(oprot)
       oprot.writeFieldEnd()
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 2)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -3307,19 +3848,22 @@ class getUserTopology_result:
   Attributes:
    - success
    - e
+   - aze
   """
 
   thrift_spec = (
     (0, TType.STRUCT, 'success', (StormTopology, StormTopology.thrift_spec), None, ), # 0
     (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 2
   )
 
   def __hash__(self):
-    return 0 + hash(self.success) + hash(self.e)
+    return 0 + hash(self.success) + hash(self.e) + hash(self.aze)
 
-  def __init__(self, success=None, e=None,):
+  def __init__(self, success=None, e=None, aze=None,):
     self.success = success
     self.e = e
+    self.aze = aze
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -3342,6 +3886,12 @@ class getUserTopology_result:
           self.e.read(iprot)
         else:
           iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -3360,6 +3910,10 @@ class getUserTopology_result:
       oprot.writeFieldBegin('e', TType.STRUCT, 1)
       self.e.write(oprot)
       oprot.writeFieldEnd()
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 2)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/py/storm/ttypes.py
----------------------------------------------------------------------
diff --git a/storm-core/src/py/storm/ttypes.py b/storm-core/src/py/storm/ttypes.py
index 2c0a50b..8fa34b0 100644
--- a/storm-core/src/py/storm/ttypes.py
+++ b/storm-core/src/py/storm/ttypes.py
@@ -1407,6 +1407,74 @@ class NotAliveException(Exception):
   def __ne__(self, other):
     return not (self == other)
 
+class AuthorizationException(Exception):
+  """
+  Attributes:
+   - msg
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'msg', None, None, ), # 1
+  )
+
+  def __hash__(self):
+    return 0 + hash(self.msg)
+
+  def __init__(self, msg=None,):
+    self.msg = msg
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.msg = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('AuthorizationException')
+    if self.msg is not None:
+      oprot.writeFieldBegin('msg', TType.STRING, 1)
+      oprot.writeString(self.msg.encode('utf-8'))
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.msg is None:
+      raise TProtocol.TProtocolException(message='Required field msg is unset!')
+    return
+
+
+  def __str__(self):
+    return repr(self)
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class InvalidTopologyException(Exception):
   """
   Attributes:
@@ -1485,6 +1553,8 @@ class TopologySummary:
    - num_workers
    - uptime_secs
    - status
+   - sched_status
+   - owner
   """
 
   thrift_spec = (
@@ -1496,12 +1566,519 @@ class TopologySummary:
     (5, TType.I32, 'num_workers', None, None, ), # 5
     (6, TType.I32, 'uptime_secs', None, None, ), # 6
     (7, TType.STRING, 'status', None, None, ), # 7
+    None, # 8
+    None, # 9
+    None, # 10
+    None, # 11
+    None, # 12
+    None, # 13
+    None, # 14
+    None, # 15
+    None, # 16
+    None, # 17
+    None, # 18
+    None, # 19
+    None, # 20
+    None, # 21
+    None, # 22
+    None, # 23
+    None, # 24
+    None, # 25
+    None, # 26
+    None, # 27
+    None, # 28
+    None, # 29
+    None, # 30
+    None, # 31
+    None, # 32
+    None, # 33
+    None, # 34
+    None, # 35
+    None, # 36
+    None, # 37
+    None, # 38
+    None, # 39
+    None, # 40
+    None, # 41
+    None, # 42
+    None, # 43
+    None, # 44
+    None, # 45
+    None, # 46
+    None, # 47
+    None, # 48
+    None, # 49
+    None, # 50
+    None, # 51
+    None, # 52
+    None, # 53
+    None, # 54
+    None, # 55
+    None, # 56
+    None, # 57
+    None, # 58
+    None, # 59
+    None, # 60
+    None, # 61
+    None, # 62
+    None, # 63
+    None, # 64
+    None, # 65
+    None, # 66
+    None, # 67
+    None, # 68
+    None, # 69
+    None, # 70
+    None, # 71
+    None, # 72
+    None, # 73
+    None, # 74
+    None, # 75
+    None, # 76
+    None, # 77
+    None, # 78
+    None, # 79
+    None, # 80
+    None, # 81
+    None, # 82
+    None, # 83
+    None, # 84
+    None, # 85
+    None, # 86
+    None, # 87
+    None, # 88
+    None, # 89
+    None, # 90
+    None, # 91
+    None, # 92
+    None, # 93
+    None, # 94
+    None, # 95
+    None, # 96
+    None, # 97
+    None, # 98
+    None, # 99
+    None, # 100
+    None, # 101
+    None, # 102
+    None, # 103
+    None, # 104
+    None, # 105
+    None, # 106
+    None, # 107
+    None, # 108
+    None, # 109
+    None, # 110
+    None, # 111
+    None, # 112
+    None, # 113
+    None, # 114
+    None, # 115
+    None, # 116
+    None, # 117
+    None, # 118
+    None, # 119
+    None, # 120
+    None, # 121
+    None, # 122
+    None, # 123
+    None, # 124
+    None, # 125
+    None, # 126
+    None, # 127
+    None, # 128
+    None, # 129
+    None, # 130
+    None, # 131
+    None, # 132
+    None, # 133
+    None, # 134
+    None, # 135
+    None, # 136
+    None, # 137
+    None, # 138
+    None, # 139
+    None, # 140
+    None, # 141
+    None, # 142
+    None, # 143
+    None, # 144
+    None, # 145
+    None, # 146
+    None, # 147
+    None, # 148
+    None, # 149
+    None, # 150
+    None, # 151
+    None, # 152
+    None, # 153
+    None, # 154
+    None, # 155
+    None, # 156
+    None, # 157
+    None, # 158
+    None, # 159
+    None, # 160
+    None, # 161
+    None, # 162
+    None, # 163
+    None, # 164
+    None, # 165
+    None, # 166
+    None, # 167
+    None, # 168
+    None, # 169
+    None, # 170
+    None, # 171
+    None, # 172
+    None, # 173
+    None, # 174
+    None, # 175
+    None, # 176
+    None, # 177
+    None, # 178
+    None, # 179
+    None, # 180
+    None, # 181
+    None, # 182
+    None, # 183
+    None, # 184
+    None, # 185
+    None, # 186
+    None, # 187
+    None, # 188
+    None, # 189
+    None, # 190
+    None, # 191
+    None, # 192
+    None, # 193
+    None, # 194
+    None, # 195
+    None, # 196
+    None, # 197
+    None, # 198
+    None, # 199
+    None, # 200
+    None, # 201
+    None, # 202
+    None, # 203
+    None, # 204
+    None, # 205
+    None, # 206
+    None, # 207
+    None, # 208
+    None, # 209
+    None, # 210
+    None, # 211
+    None, # 212
+    None, # 213
+    None, # 214
+    None, # 215
+    None, # 216
+    None, # 217
+    None, # 218
+    None, # 219
+    None, # 220
+    None, # 221
+    None, # 222
+    None, # 223
+    None, # 224
+    None, # 225
+    None, # 226
+    None, # 227
+    None, # 228
+    None, # 229
+    None, # 230
+    None, # 231
+    None, # 232
+    None, # 233
+    None, # 234
+    None, # 235
+    None, # 236
+    None, # 237
+    None, # 238
+    None, # 239
+    None, # 240
+    None, # 241
+    None, # 242
+    None, # 243
+    None, # 244
+    None, # 245
+    None, # 246
+    None, # 247
+    None, # 248
+    None, # 249
+    None, # 250
+    None, # 251
+    None, # 252
+    None, # 253
+    None, # 254
+    None, # 255
+    None, # 256
+    None, # 257
+    None, # 258
+    None, # 259
+    None, # 260
+    None, # 261
+    None, # 262
+    None, # 263
+    None, # 264
+    None, # 265
+    None, # 266
+    None, # 267
+    None, # 268
+    None, # 269
+    None, # 270
+    None, # 271
+    None, # 272
+    None, # 273
+    None, # 274
+    None, # 275
+    None, # 276
+    None, # 277
+    None, # 278
+    None, # 279
+    None, # 280
+    None, # 281
+    None, # 282
+    None, # 283
+    None, # 284
+    None, # 285
+    None, # 286
+    None, # 287
+    None, # 288
+    None, # 289
+    None, # 290
+    None, # 291
+    None, # 292
+    None, # 293
+    None, # 294
+    None, # 295
+    None, # 296
+    None, # 297
+    None, # 298
+    None, # 299
+    None, # 300
+    None, # 301
+    None, # 302
+    None, # 303
+    None, # 304
+    None, # 305
+    None, # 306
+    None, # 307
+    None, # 308
+    None, # 309
+    None, # 310
+    None, # 311
+    None, # 312
+    None, # 313
+    None, # 314
+    None, # 315
+    None, # 316
+    None, # 317
+    None, # 318
+    None, # 319
+    None, # 320
+    None, # 321
+    None, # 322
+    None, # 323
+    None, # 324
+    None, # 325
+    None, # 326
+    None, # 327
+    None, # 328
+    None, # 329
+    None, # 330
+    None, # 331
+    None, # 332
+    None, # 333
+    None, # 334
+    None, # 335
+    None, # 336
+    None, # 337
+    None, # 338
+    None, # 339
+    None, # 340
+    None, # 341
+    None, # 342
+    None, # 343
+    None, # 344
+    None, # 345
+    None, # 346
+    None, # 347
+    None, # 348
+    None, # 349
+    None, # 350
+    None, # 351
+    None, # 352
+    None, # 353
+    None, # 354
+    None, # 355
+    None, # 356
+    None, # 357
+    None, # 358
+    None, # 359
+    None, # 360
+    None, # 361
+    None, # 362
+    None, # 363
+    None, # 364
+    None, # 365
+    None, # 366
+    None, # 367
+    None, # 368
+    None, # 369
+    None, # 370
+    None, # 371
+    None, # 372
+    None, # 373
+    None, # 374
+    None, # 375
+    None, # 376
+    None, # 377
+    None, # 378
+    None, # 379
+    None, # 380
+    None, # 381
+    None, # 382
+    None, # 383
+    None, # 384
+    None, # 385
+    None, # 386
+    None, # 387
+    None, # 388
+    None, # 389
+    None, # 390
+    None, # 391
+    None, # 392
+    None, # 393
+    None, # 394
+    None, # 395
+    None, # 396
+    None, # 397
+    None, # 398
+    None, # 399
+    None, # 400
+    None, # 401
+    None, # 402
+    None, # 403
+    None, # 404
+    None, # 405
+    None, # 406
+    None, # 407
+    None, # 408
+    None, # 409
+    None, # 410
+    None, # 411
+    None, # 412
+    None, # 413
+    None, # 414
+    None, # 415
+    None, # 416
+    None, # 417
+    None, # 418
+    None, # 419
+    None, # 420
+    None, # 421
+    None, # 422
+    None, # 423
+    None, # 424
+    None, # 425
+    None, # 426
+    None, # 427
+    None, # 428
+    None, # 429
+    None, # 430
+    None, # 431
+    None, # 432
+    None, # 433
+    None, # 434
+    None, # 435
+    None, # 436
+    None, # 437
+    None, # 438
+    None, # 439
+    None, # 440
+    None, # 441
+    None, # 442
+    None, # 443
+    None, # 444
+    None, # 445
+    None, # 446
+    None, # 447
+    None, # 448
+    None, # 449
+    None, # 450
+    None, # 451
+    None, # 452
+    None, # 453
+    None, # 454
+    None, # 455
+    None, # 456
+    None, # 457
+    None, # 458
+    None, # 459
+    None, # 460
+    None, # 461
+    None, # 462
+    None, # 463
+    None, # 464
+    None, # 465
+    None, # 466
+    None, # 467
+    None, # 468
+    None, # 469
+    None, # 470
+    None, # 471
+    None, # 472
+    None, # 473
+    None, # 474
+    None, # 475
+    None, # 476
+    None, # 477
+    None, # 478
+    None, # 479
+    None, # 480
+    None, # 481
+    None, # 482
+    None, # 483
+    None, # 484
+    None, # 485
+    None, # 486
+    None, # 487
+    None, # 488
+    None, # 489
+    None, # 490
+    None, # 491
+    None, # 492
+    None, # 493
+    None, # 494
+    None, # 495
+    None, # 496
+    None, # 497
+    None, # 498
+    None, # 499
+    None, # 500
+    None, # 501
+    None, # 502
+    None, # 503
+    None, # 504
+    None, # 505
+    None, # 506
+    None, # 507
+    None, # 508
+    None, # 509
+    None, # 510
+    None, # 511
+    None, # 512
+    (513, TType.STRING, 'sched_status', None, None, ), # 513
+    (514, TType.STRING, 'owner', None, None, ), # 514
   )
 
   def __hash__(self):
-    return 0 + hash(self.id) + hash(self.name) + hash(self.num_tasks) + hash(self.num_executors) + hash(self.num_workers) + hash(self.uptime_secs) + hash(self.status)
+    return 0 + hash(self.id) + hash(self.name) + hash(self.num_tasks) + hash(self.num_executors) + hash(self.num_workers) + hash(self.uptime_secs) + hash(self.status) + hash(self.sched_status) + hash(self.owner)
 
-  def __init__(self, id=None, name=None, num_tasks=None, num_executors=None, num_workers=None, uptime_secs=None, status=None,):
+  def __init__(self, id=None, name=None, num_tasks=None, num_executors=None, num_workers=None, uptime_secs=None, status=None, sched_status=None, owner=None,):
     self.id = id
     self.name = name
     self.num_tasks = num_tasks
@@ -1509,6 +2086,8 @@ class TopologySummary:
     self.num_workers = num_workers
     self.uptime_secs = uptime_secs
     self.status = status
+    self.sched_status = sched_status
+    self.owner = owner
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -1554,6 +2133,16 @@ class TopologySummary:
           self.status = iprot.readString().decode('utf-8')
         else:
           iprot.skip(ftype)
+      elif fid == 513:
+        if ftype == TType.STRING:
+          self.sched_status = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 514:
+        if ftype == TType.STRING:
+          self.owner = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -1592,6 +2181,14 @@ class TopologySummary:
       oprot.writeFieldBegin('status', TType.STRING, 7)
       oprot.writeString(self.status.encode('utf-8'))
       oprot.writeFieldEnd()
+    if self.sched_status is not None:
+      oprot.writeFieldBegin('sched_status', TType.STRING, 513)
+      oprot.writeString(self.sched_status.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.owner is not None:
+      oprot.writeFieldBegin('owner', TType.STRING, 514)
+      oprot.writeString(self.owner.encode('utf-8'))
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -2749,6 +3346,8 @@ class TopologyInfo:
    - executors
    - status
    - errors
+   - sched_status
+   - owner
   """
 
   thrift_spec = (
@@ -2759,18 +3358,528 @@ class TopologyInfo:
     (4, TType.LIST, 'executors', (TType.STRUCT,(ExecutorSummary, ExecutorSummary.thrift_spec)), None, ), # 4
     (5, TType.STRING, 'status', None, None, ), # 5
     (6, TType.MAP, 'errors', (TType.STRING,None,TType.LIST,(TType.STRUCT,(ErrorInfo, ErrorInfo.thrift_spec))), None, ), # 6
+    None, # 7
+    None, # 8
+    None, # 9
+    None, # 10
+    None, # 11
+    None, # 12
+    None, # 13
+    None, # 14
+    None, # 15
+    None, # 16
+    None, # 17
+    None, # 18
+    None, # 19
+    None, # 20
+    None, # 21
+    None, # 22
+    None, # 23
+    None, # 24
+    None, # 25
+    None, # 26
+    None, # 27
+    None, # 28
+    None, # 29
+    None, # 30
+    None, # 31
+    None, # 32
+    None, # 33
+    None, # 34
+    None, # 35
+    None, # 36
+    None, # 37
+    None, # 38
+    None, # 39
+    None, # 40
+    None, # 41
+    None, # 42
+    None, # 43
+    None, # 44
+    None, # 45
+    None, # 46
+    None, # 47
+    None, # 48
+    None, # 49
+    None, # 50
+    None, # 51
+    None, # 52
+    None, # 53
+    None, # 54
+    None, # 55
+    None, # 56
+    None, # 57
+    None, # 58
+    None, # 59
+    None, # 60
+    None, # 61
+    None, # 62
+    None, # 63
+    None, # 64
+    None, # 65
+    None, # 66
+    None, # 67
+    None, # 68
+    None, # 69
+    None, # 70
+    None, # 71
+    None, # 72
+    None, # 73
+    None, # 74
+    None, # 75
+    None, # 76
+    None, # 77
+    None, # 78
+    None, # 79
+    None, # 80
+    None, # 81
+    None, # 82
+    None, # 83
+    None, # 84
+    None, # 85
+    None, # 86
+    None, # 87
+    None, # 88
+    None, # 89
+    None, # 90
+    None, # 91
+    None, # 92
+    None, # 93
+    None, # 94
+    None, # 95
+    None, # 96
+    None, # 97
+    None, # 98
+    None, # 99
+    None, # 100
+    None, # 101
+    None, # 102
+    None, # 103
+    None, # 104
+    None, # 105
+    None, # 106
+    None, # 107
+    None, # 108
+    None, # 109
+    None, # 110
+    None, # 111
+    None, # 112
+    None, # 113
+    None, # 114
+    None, # 115
+    None, # 116
+    None, # 117
+    None, # 118
+    None, # 119
+    None, # 120
+    None, # 121
+    None, # 122
+    None, # 123
+    None, # 124
+    None, # 125
+    None, # 126
+    None, # 127
+    None, # 128
+    None, # 129
+    None, # 130
+    None, # 131
+    None, # 132
+    None, # 133
+    None, # 134
+    None, # 135
+    None, # 136
+    None, # 137
+    None, # 138
+    None, # 139
+    None, # 140
+    None, # 141
+    None, # 142
+    None, # 143
+    None, # 144
+    None, # 145
+    None, # 146
+    None, # 147
+    None, # 148
+    None, # 149
+    None, # 150
+    None, # 151
+    None, # 152
+    None, # 153
+    None, # 154
+    None, # 155
+    None, # 156
+    None, # 157
+    None, # 158
+    None, # 159
+    None, # 160
+    None, # 161
+    None, # 162
+    None, # 163
+    None, # 164
+    None, # 165
+    None, # 166
+    None, # 167
+    None, # 168
+    None, # 169
+    None, # 170
+    None, # 171
+    None, # 172
+    None, # 173
+    None, # 174
+    None, # 175
+    None, # 176
+    None, # 177
+    None, # 178
+    None, # 179
+    None, # 180
+    None, # 181
+    None, # 182
+    None, # 183
+    None, # 184
+    None, # 185
+    None, # 186
+    None, # 187
+    None, # 188
+    None, # 189
+    None, # 190
+    None, # 191
+    None, # 192
+    None, # 193
+    None, # 194
+    None, # 195
+    None, # 196
+    None, # 197
+    None, # 198
+    None, # 199
+    None, # 200
+    None, # 201
+    None, # 202
+    None, # 203
+    None, # 204
+    None, # 205
+    None, # 206
+    None, # 207
+    None, # 208
+    None, # 209
+    None, # 210
+    None, # 211
+    None, # 212
+    None, # 213
+    None, # 214
+    None, # 215
+    None, # 216
+    None, # 217
+    None, # 218
+    None, # 219
+    None, # 220
+    None, # 221
+    None, # 222
+    None, # 223
+    None, # 224
+    None, # 225
+    None, # 226
+    None, # 227
+    None, # 228
+    None, # 229
+    None, # 230
+    None, # 231
+    None, # 232
+    None, # 233
+    None, # 234
+    None, # 235
+    None, # 236
+    None, # 237
+    None, # 238
+    None, # 239
+    None, # 240
+    None, # 241
+    None, # 242
+    None, # 243
+    None, # 244
+    None, # 245
+    None, # 246
+    None, # 247
+    None, # 248
+    None, # 249
+    None, # 250
+    None, # 251
+    None, # 252
+    None, # 253
+    None, # 254
+    None, # 255
+    None, # 256
+    None, # 257
+    None, # 258
+    None, # 259
+    None, # 260
+    None, # 261
+    None, # 262
+    None, # 263
+    None, # 264
+    None, # 265
+    None, # 266
+    None, # 267
+    None, # 268
+    None, # 269
+    None, # 270
+    None, # 271
+    None, # 272
+    None, # 273
+    None, # 274
+    None, # 275
+    None, # 276
+    None, # 277
+    None, # 278
+    None, # 279
+    None, # 280
+    None, # 281
+    None, # 282
+    None, # 283
+    None, # 284
+    None, # 285
+    None, # 286
+    None, # 287
+    None, # 288
+    None, # 289
+    None, # 290
+    None, # 291
+    None, # 292
+    None, # 293
+    None, # 294
+    None, # 295
+    None, # 296
+    None, # 297
+    None, # 298
+    None, # 299
+    None, # 300
+    None, # 301
+    None, # 302
+    None, # 303
+    None, # 304
+    None, # 305
+    None, # 306
+    None, # 307
+    None, # 308
+    None, # 309
+    None, # 310
+    None, # 311
+    None, # 312
+    None, # 313
+    None, # 314
+    None, # 315
+    None, # 316
+    None, # 317
+    None, # 318
+    None, # 319
+    None, # 320
+    None, # 321
+    None, # 322
+    None, # 323
+    None, # 324
+    None, # 325
+    None, # 326
+    None, # 327
+    None, # 328
+    None, # 329
+    None, # 330
+    None, # 331
+    None, # 332
+    None, # 333
+    None, # 334
+    None, # 335
+    None, # 336
+    None, # 337
+    None, # 338
+    None, # 339
+    None, # 340
+    None, # 341
+    None, # 342
+    None, # 343
+    None, # 344
+    None, # 345
+    None, # 346
+    None, # 347
+    None, # 348
+    None, # 349
+    None, # 350
+    None, # 351
+    None, # 352
+    None, # 353
+    None, # 354
+    None, # 355
+    None, # 356
+    None, # 357
+    None, # 358
+    None, # 359
+    None, # 360
+    None, # 361
+    None, # 362
+    None, # 363
+    None, # 364
+    None, # 365
+    None, # 366
+    None, # 367
+    None, # 368
+    None, # 369
+    None, # 370
+    None, # 371
+    None, # 372
+    None, # 373
+    None, # 374
+    None, # 375
+    None, # 376
+    None, # 377
+    None, # 378
+    None, # 379
+    None, # 380
+    None, # 381
+    None, # 382
+    None, # 383
+    None, # 384
+    None, # 385
+    None, # 386
+    None, # 387
+    None, # 388
+    None, # 389
+    None, # 390
+    None, # 391
+    None, # 392
+    None, # 393
+    None, # 394
+    None, # 395
+    None, # 396
+    None, # 397
+    None, # 398
+    None, # 399
+    None, # 400
+    None, # 401
+    None, # 402
+    None, # 403
+    None, # 404
+    None, # 405
+    None, # 406
+    None, # 407
+    None, # 408
+    None, # 409
+    None, # 410
+    None, # 411
+    None, # 412
+    None, # 413
+    None, # 414
+    None, # 415
+    None, # 416
+    None, # 417
+    None, # 418
+    None, # 419
+    None, # 420
+    None, # 421
+    None, # 422
+    None, # 423
+    None, # 424
+    None, # 425
+    None, # 426
+    None, # 427
+    None, # 428
+    None, # 429
+    None, # 430
+    None, # 431
+    None, # 432
+    None, # 433
+    None, # 434
+    None, # 435
+    None, # 436
+    None, # 437
+    None, # 438
+    None, # 439
+    None, # 440
+    None, # 441
+    None, # 442
+    None, # 443
+    None, # 444
+    None, # 445
+    None, # 446
+    None, # 447
+    None, # 448
+    None, # 449
+    None, # 450
+    None, # 451
+    None, # 452
+    None, # 453
+    None, # 454
+    None, # 455
+    None, # 456
+    None, # 457
+    None, # 458
+    None, # 459
+    None, # 460
+    None, # 461
+    None, # 462
+    None, # 463
+    None, # 464
+    None, # 465
+    None, # 466
+    None, # 467
+    None, # 468
+    None, # 469
+    None, # 470
+    None, # 471
+    None, # 472
+    None, # 473
+    None, # 474
+    None, # 475
+    None, # 476
+    None, # 477
+    None, # 478
+    None, # 479
+    None, # 480
+    None, # 481
+    None, # 482
+    None, # 483
+    None, # 484
+    None, # 485
+    None, # 486
+    None, # 487
+    None, # 488
+    None, # 489
+    None, # 490
+    None, # 491
+    None, # 492
+    None, # 493
+    None, # 494
+    None, # 495
+    None, # 496
+    None, # 497
+    None, # 498
+    None, # 499
+    None, # 500
+    None, # 501
+    None, # 502
+    None, # 503
+    None, # 504
+    None, # 505
+    None, # 506
+    None, # 507
+    None, # 508
+    None, # 509
+    None, # 510
+    None, # 511
+    None, # 512
+    (513, TType.STRING, 'sched_status', None, None, ), # 513
+    (514, TType.STRING, 'owner', None, None, ), # 514
   )
 
   def __hash__(self):
-    return 0 + hash(self.id) + hash(self.name) + hash(self.uptime_secs) + hash(self.executors) + hash(self.status) + hash(self.errors)
+    return 0 + hash(self.id) + hash(self.name) + hash(self.uptime_secs) + hash(self.executors) + hash(self.status) + hash(self.errors) + hash(self.sched_status) + hash(self.owner)
 
-  def __init__(self, id=None, name=None, uptime_secs=None, executors=None, status=None, errors=None,):
+  def __init__(self, id=None, name=None, uptime_secs=None, executors=None, status=None, errors=None, sched_status=None, owner=None,):
     self.id = id
     self.name = name
     self.uptime_secs = uptime_secs
     self.executors = executors
     self.status = status
     self.errors = errors
+    self.sched_status = sched_status
+    self.owner = owner
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -2829,6 +3938,16 @@ class TopologyInfo:
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
+      elif fid == 513:
+        if ftype == TType.STRING:
+          self.sched_status = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 514:
+        if ftype == TType.STRING:
+          self.owner = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -2873,6 +3992,14 @@ class TopologyInfo:
         oprot.writeListEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
+    if self.sched_status is not None:
+      oprot.writeFieldBegin('sched_status', TType.STRING, 513)
+      oprot.writeString(self.sched_status.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.owner is not None:
+      oprot.writeFieldBegin('owner', TType.STRING, 514)
+      oprot.writeString(self.owner.encode('utf-8'))
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -3063,22 +4190,100 @@ class RebalanceOptions:
   def __ne__(self, other):
     return not (self == other)
 
+class Credentials:
+  """
+  Attributes:
+   - creds
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.MAP, 'creds', (TType.STRING,None,TType.STRING,None), None, ), # 1
+  )
+
+  def __hash__(self):
+    return 0 + hash(self.creds)
+
+  def __init__(self, creds=None,):
+    self.creds = creds
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.MAP:
+          self.creds = {}
+          (_ktype293, _vtype294, _size292 ) = iprot.readMapBegin() 
+          for _i296 in xrange(_size292):
+            _key297 = iprot.readString().decode('utf-8')
+            _val298 = iprot.readString().decode('utf-8')
+            self.creds[_key297] = _val298
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('Credentials')
+    if self.creds is not None:
+      oprot.writeFieldBegin('creds', TType.MAP, 1)
+      oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.creds))
+      for kiter299,viter300 in self.creds.items():
+        oprot.writeString(kiter299.encode('utf-8'))
+        oprot.writeString(viter300.encode('utf-8'))
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.creds is None:
+      raise TProtocol.TProtocolException(message='Required field creds is unset!')
+    return
+
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class SubmitOptions:
   """
   Attributes:
    - initial_status
+   - creds
   """
 
   thrift_spec = (
     None, # 0
     (1, TType.I32, 'initial_status', None, None, ), # 1
+    (2, TType.STRUCT, 'creds', (Credentials, Credentials.thrift_spec), None, ), # 2
   )
 
   def __hash__(self):
-    return 0 + hash(self.initial_status)
+    return 0 + hash(self.initial_status) + hash(self.creds)
 
-  def __init__(self, initial_status=None,):
+  def __init__(self, initial_status=None, creds=None,):
     self.initial_status = initial_status
+    self.creds = creds
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -3094,6 +4299,12 @@ class SubmitOptions:
           self.initial_status = iprot.readI32();
         else:
           iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.creds = Credentials()
+          self.creds.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -3108,6 +4319,10 @@ class SubmitOptions:
       oprot.writeFieldBegin('initial_status', TType.I32, 1)
       oprot.writeI32(self.initial_status)
       oprot.writeFieldEnd()
+    if self.creds is not None:
+      oprot.writeFieldBegin('creds', TType.STRUCT, 2)
+      self.creds.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/storm.thrift
----------------------------------------------------------------------
diff --git a/storm-core/src/storm.thrift b/storm-core/src/storm.thrift
index 3c54d30..679d29e 100644
--- a/storm-core/src/storm.thrift
+++ b/storm-core/src/storm.thrift
@@ -127,6 +127,10 @@ exception NotAliveException {
   1: required string msg;
 }
 
+exception AuthorizationException {
+  1: required string msg;
+}
+
 exception InvalidTopologyException {
   1: required string msg;
 }
@@ -139,6 +143,8 @@ struct TopologySummary {
   5: required i32 num_workers;
   6: required i32 uptime_secs;
   7: required string status;
+513: optional string sched_status;
+514: optional string owner;
 }
 
 struct SupervisorSummary {
@@ -208,6 +214,8 @@ struct TopologyInfo {
   4: required list<ExecutorSummary> executors;
   5: required string status;
   6: required map<string, list<ErrorInfo>> errors;
+513: optional string sched_status;
+514: optional string owner;
 }
 
 struct KillOptions {
@@ -220,42 +228,48 @@ struct RebalanceOptions {
   3: optional map<string, i32> num_executors;
 }
 
+struct Credentials {
+  1: required map<string,string> creds;
+}
+
 enum TopologyInitialStatus {
     ACTIVE = 1,
     INACTIVE = 2
 }
 struct SubmitOptions {
   1: required TopologyInitialStatus initial_status;
+  2: optional Credentials creds;
 }
 
 service Nimbus {
-  void submitTopology(1: string name, 2: string uploadedJarLocation, 3: string jsonConf, 4: StormTopology topology) throws (1: AlreadyAliveException e, 2: InvalidTopologyException ite);
-  void submitTopologyWithOpts(1: string name, 2: string uploadedJarLocation, 3: string jsonConf, 4: StormTopology topology, 5: SubmitOptions options) throws (1: AlreadyAliveException e, 2: InvalidTopologyException ite);
-  void killTopology(1: string name) throws (1: NotAliveException e);
-  void killTopologyWithOpts(1: string name, 2: KillOptions options) throws (1: NotAliveException e);
-  void activate(1: string name) throws (1: NotAliveException e);
-  void deactivate(1: string name) throws (1: NotAliveException e);
-  void rebalance(1: string name, 2: RebalanceOptions options) throws (1: NotAliveException e, 2: InvalidTopologyException ite);
+  void submitTopology(1: string name, 2: string uploadedJarLocation, 3: string jsonConf, 4: StormTopology topology) throws (1: AlreadyAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze);
+  void submitTopologyWithOpts(1: string name, 2: string uploadedJarLocation, 3: string jsonConf, 4: StormTopology topology, 5: SubmitOptions options) throws (1: AlreadyAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze);
+  void killTopology(1: string name) throws (1: NotAliveException e, 2: AuthorizationException aze);
+  void killTopologyWithOpts(1: string name, 2: KillOptions options) throws (1: NotAliveException e, 2: AuthorizationException aze);
+  void activate(1: string name) throws (1: NotAliveException e, 2: AuthorizationException aze);
+  void deactivate(1: string name) throws (1: NotAliveException e, 2: AuthorizationException aze);
+  void rebalance(1: string name, 2: RebalanceOptions options) throws (1: NotAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze);
+  void uploadNewCredentials(1: string name, 2: Credentials creds) throws (1: NotAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze);
 
   // need to add functions for asking about status of storms, what nodes they're running on, looking at task logs
 
-  string beginFileUpload();
-  void uploadChunk(1: string location, 2: binary chunk);
-  void finishFileUpload(1: string location);
+  string beginFileUpload() throws (1: AuthorizationException aze);
+  void uploadChunk(1: string location, 2: binary chunk) throws (1: AuthorizationException aze);
+  void finishFileUpload(1: string location) throws (1: AuthorizationException aze);
   
-  string beginFileDownload(1: string file);
+  string beginFileDownload(1: string file) throws (1: AuthorizationException aze);
   //can stop downloading chunks when receive 0-length byte array back
-  binary downloadChunk(1: string id);
+  binary downloadChunk(1: string id) throws (1: AuthorizationException aze);
 
   // returns json
-  string getNimbusConf();
+  string getNimbusConf() throws (1: AuthorizationException aze);
   // stats functions
-  ClusterSummary getClusterInfo();
-  TopologyInfo getTopologyInfo(1: string id) throws (1: NotAliveException e);
+  ClusterSummary getClusterInfo() throws (1: AuthorizationException aze);
+  TopologyInfo getTopologyInfo(1: string id) throws (1: NotAliveException e, 2: AuthorizationException aze);
   //returns json
-  string getTopologyConf(1: string id) throws (1: NotAliveException e);
-  StormTopology getTopology(1: string id) throws (1: NotAliveException e);
-  StormTopology getUserTopology(1: string id) throws (1: NotAliveException e);
+  string getTopologyConf(1: string id) throws (1: NotAliveException e, 2: AuthorizationException aze);
+  StormTopology getTopology(1: string id) throws (1: NotAliveException e, 2: AuthorizationException aze);
+  StormTopology getUserTopology(1: string id) throws (1: NotAliveException e, 2: AuthorizationException aze);
 }
 
 struct DRPCRequest {
@@ -268,11 +282,11 @@ exception DRPCExecutionException {
 }
 
 service DistributedRPC {
-  string execute(1: string functionName, 2: string funcArgs) throws (1: DRPCExecutionException e);
+  string execute(1: string functionName, 2: string funcArgs) throws (1: DRPCExecutionException e, 2: AuthorizationException aze);
 }
 
 service DistributedRPCInvocations {
-  void result(1: string id, 2: string result);
-  DRPCRequest fetchRequest(1: string functionName);
-  void failRequest(1: string id);  
+  void result(1: string id, 2: string result) throws (1: AuthorizationException aze);
+  DRPCRequest fetchRequest(1: string functionName) throws (1: AuthorizationException aze);
+  void failRequest(1: string id) throws (1: AuthorizationException aze);  
 }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/fe5f41aa/storm-core/src/ui/public/css/style.css
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/css/style.css b/storm-core/src/ui/public/css/style.css
index 6c7c119..29a45eb 100644
--- a/storm-core/src/ui/public/css/style.css
+++ b/storm-core/src/ui/public/css/style.css
@@ -19,8 +19,14 @@
     display: none;
 }
 
+.ui-user {
+    float: right;
+    padding: 0.5em;
+}
+
 body {
   color: #808080;
+  padding: 0.2em;
 }
 
 table {