Repository: storm Updated Branches: refs/heads/master 872d0a13c -> 2832aeb36
http://git-wip-us.apache.org/repos/asf/storm/blob/725003cc/storm-core/test/clj/org/apache/storm/scheduler/multitenant_scheduler_test.clj ---------------------------------------------------------------------- diff --git a/storm-core/test/clj/org/apache/storm/scheduler/multitenant_scheduler_test.clj b/storm-core/test/clj/org/apache/storm/scheduler/multitenant_scheduler_test.clj index 0c3ee9b..c7b5050 100644 --- a/storm-core/test/clj/org/apache/storm/scheduler/multitenant_scheduler_test.clj +++ b/storm-core/test/clj/org/apache/storm/scheduler/multitenant_scheduler_test.clj @@ -16,8 +16,8 @@ (ns org.apache.storm.scheduler.multitenant-scheduler-test (:use [clojure test]) (:use [org.apache.storm config testing log]) - (:require [org.apache.storm.daemon [nimbus :as nimbus]]) (:import [org.apache.storm.generated StormTopology]) + (:import [org.apache.storm.daemon.nimbus Nimbus$StandaloneINimbus]) (:import [org.apache.storm.scheduler Cluster SupervisorDetails WorkerSlot ExecutorDetails SchedulerAssignmentImpl Topologies TopologyDetails]) (:import [org.apache.storm.scheduler.multitenant Node NodePool FreePool DefaultPool @@ -42,7 +42,7 @@ (deftest test-node (let [supers (gen-supervisors 5) - cluster (Cluster. (nimbus/standalone-nimbus) supers {} nil) + cluster (Cluster. (Nimbus$StandaloneINimbus.) supers {} nil) node-map (Node/getAllNodesFrom cluster)] (is (= 5 (.size node-map))) (let [node (.get node-map "super0")] @@ -87,7 +87,7 @@ (deftest test-free-pool (let [supers (gen-supervisors 5) - cluster (Cluster. (nimbus/standalone-nimbus) supers {} nil) + cluster (Cluster. (Nimbus$StandaloneINimbus.) supers {} nil) node-map (Node/getAllNodesFrom cluster) free-pool (FreePool. )] ;; assign one node so it is not in the pool @@ -125,7 +125,7 @@ (deftest test-default-pool-simple (let [supers (gen-supervisors 5) - cluster (Cluster. (nimbus/standalone-nimbus) supers {} nil) + cluster (Cluster. (Nimbus$StandaloneINimbus.) supers {} nil) node-map (Node/getAllNodesFrom cluster) free-pool (FreePool. ) default-pool (DefaultPool. ) @@ -162,7 +162,7 @@ (deftest test-default-pool-big-request (let [supers (gen-supervisors 5) - cluster (Cluster. (nimbus/standalone-nimbus) supers {} nil) + cluster (Cluster. (Nimbus$StandaloneINimbus.) supers {} nil) node-map (Node/getAllNodesFrom cluster) free-pool (FreePool. ) default-pool (DefaultPool. ) @@ -199,7 +199,7 @@ (deftest test-default-pool-big-request-2 (let [supers (gen-supervisors 1) - cluster (Cluster. (nimbus/standalone-nimbus) supers {} nil) + cluster (Cluster. (Nimbus$StandaloneINimbus.) supers {} nil) node-map (Node/getAllNodesFrom cluster) free-pool (FreePool. ) default-pool (DefaultPool. ) @@ -241,7 +241,7 @@ (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 {} nil) + single-cluster (Cluster. (Nimbus$StandaloneINimbus.) single-super {} nil) executor1 (ed 1) executor2 (ed 2) executor3 (ed 3) @@ -268,7 +268,7 @@ (is (= "Running with fewer slots than requested (4/5)" (.get (.getStatusMap single-cluster) "topology1"))) ) - (let [cluster (Cluster. (nimbus/standalone-nimbus) supers (.getAssignments single-cluster) nil) + (let [cluster (Cluster. (Nimbus$StandaloneINimbus.) supers (.getAssignments single-cluster) nil) node-map (Node/getAllNodesFrom cluster) free-pool (FreePool. ) default-pool (DefaultPool. )] @@ -285,7 +285,7 @@ (deftest test-default-pool-complex (let [supers (gen-supervisors 5) - cluster (Cluster. (nimbus/standalone-nimbus) supers {} nil) + cluster (Cluster. (Nimbus$StandaloneINimbus.) supers {} nil) node-map (Node/getAllNodesFrom cluster) free-pool (FreePool. ) default-pool (DefaultPool. ) @@ -367,7 +367,7 @@ (deftest test-isolated-pool-simple (let [supers (gen-supervisors 5) - cluster (Cluster. (nimbus/standalone-nimbus) supers {} nil) + cluster (Cluster. (Nimbus$StandaloneINimbus.) supers {} nil) node-map (Node/getAllNodesFrom cluster) free-pool (FreePool. ) isolated-pool (IsolatedPool. 5) @@ -411,7 +411,7 @@ (deftest test-isolated-pool-big-ask (let [supers (gen-supervisors 5) - cluster (Cluster. (nimbus/standalone-nimbus) supers {} nil) + cluster (Cluster. (Nimbus$StandaloneINimbus.) supers {} nil) node-map (Node/getAllNodesFrom cluster) free-pool (FreePool. ) isolated-pool (IsolatedPool. 5) @@ -455,7 +455,7 @@ (deftest test-isolated-pool-complex (let [supers (gen-supervisors 5) - cluster (Cluster. (nimbus/standalone-nimbus) supers {} nil) + cluster (Cluster. (Nimbus$StandaloneINimbus.) supers {} nil) node-map (Node/getAllNodesFrom cluster) free-pool (FreePool. ) isolated-pool (IsolatedPool. 5) @@ -559,7 +559,7 @@ (deftest test-isolated-pool-complex-2 (let [supers (gen-supervisors 5) - cluster (Cluster. (nimbus/standalone-nimbus) supers {} nil) + cluster (Cluster. (Nimbus$StandaloneINimbus.) supers {} nil) node-map (Node/getAllNodesFrom cluster) free-pool (FreePool. ) ;;like before but now we can only hold 2 nodes max. Don't go over @@ -657,7 +657,7 @@ ["bolt22" 10 20] ["bolt23" 20 30] ["bolt24" 30 40]])) - cluster (Cluster. (nimbus/standalone-nimbus) supers {} nil) + cluster (Cluster. (Nimbus$StandaloneINimbus.) supers {} nil) node-map (Node/getAllNodesFrom cluster) topologies (Topologies. (to-top-map [topology1 topology2 topology3])) conf {MULTITENANT-SCHEDULER-USER-POOLS {"userA" 5 "userB" 5}} @@ -696,7 +696,7 @@ (ExecutorDetails. 10 15) (WorkerSlot. "super0" 1) (ExecutorDetails. 15 20) (WorkerSlot. "super0" 1)}) } - cluster (Cluster. (nimbus/standalone-nimbus) supers existing-assignments nil) + cluster (Cluster. (Nimbus$StandaloneINimbus.) supers existing-assignments nil) node-map (Node/getAllNodesFrom cluster) topologies (Topologies. (to-top-map [topology1])) conf {MULTITENANT-SCHEDULER-USER-POOLS {"userA" 5 "userB" 5}} @@ -741,7 +741,7 @@ worker-slot-with-multiple-assignments (WorkerSlot. "super1" 1) existing-assignments {"topology2" (SchedulerAssignmentImpl. "topology2" {(ExecutorDetails. 1 1) worker-slot-with-multiple-assignments}) "topology3" (SchedulerAssignmentImpl. "topology3" {(ExecutorDetails. 2 2) worker-slot-with-multiple-assignments})} - cluster (Cluster. (nimbus/standalone-nimbus) supers existing-assignments nil) + cluster (Cluster. (Nimbus$StandaloneINimbus.) supers existing-assignments nil) topologies (Topologies. (to-top-map [topology1 topology2 topology3])) conf {MULTITENANT-SCHEDULER-USER-POOLS {"userA" 2 "userB" 1}} scheduler (MultitenantScheduler.)] @@ -769,7 +769,7 @@ existing-assignments {"topology1" (SchedulerAssignmentImpl. "topology1" {(ExecutorDetails. 0 0) (WorkerSlot. "super0" port-not-reported-by-supervisor)})} - cluster (Cluster. (nimbus/standalone-nimbus) supers existing-assignments nil) + cluster (Cluster. (Nimbus$StandaloneINimbus.) supers existing-assignments nil) topologies (Topologies. (to-top-map [topology1])) conf {} scheduler (MultitenantScheduler.)] @@ -809,7 +809,7 @@ (SchedulerAssignmentImpl. "topology2" {(ExecutorDetails. 4 4) worker-slot-with-multiple-assignments (ExecutorDetails. 5 5) (WorkerSlot. dead-supervisor port-not-reported-by-supervisor)})} - cluster (Cluster. (nimbus/standalone-nimbus) supers existing-assignments nil) + cluster (Cluster. (Nimbus$StandaloneINimbus.) supers existing-assignments nil) topologies (Topologies. (to-top-map [topology1 topology2])) conf {} scheduler (MultitenantScheduler.)] @@ -850,7 +850,7 @@ (ExecutorDetails. 3 3) (WorkerSlot. "super1" 4) (ExecutorDetails. 4 4) (WorkerSlot. "super2" 1) (ExecutorDetails. 5 5) (WorkerSlot. "super2" 2)})} - cluster (Cluster. (nimbus/standalone-nimbus) supers existing-assignments nil) + cluster (Cluster. (Nimbus$StandaloneINimbus.) supers existing-assignments nil) topologies (Topologies. (to-top-map [topology1])) conf {MULTITENANT-SCHEDULER-USER-POOLS {"userA" 2}} scheduler (MultitenantScheduler.)] http://git-wip-us.apache.org/repos/asf/storm/blob/725003cc/storm-core/test/clj/org/apache/storm/scheduler_test.clj ---------------------------------------------------------------------- diff --git a/storm-core/test/clj/org/apache/storm/scheduler_test.clj b/storm-core/test/clj/org/apache/storm/scheduler_test.clj index 430702e..003c650 100644 --- a/storm-core/test/clj/org/apache/storm/scheduler_test.clj +++ b/storm-core/test/clj/org/apache/storm/scheduler_test.clj @@ -17,7 +17,7 @@ (:use [clojure test]) (:use [org.apache.storm util config testing]) (:import [org.apache.storm.scheduler EvenScheduler]) - (:require [org.apache.storm.daemon [nimbus :as nimbus]]) + (:import [org.apache.storm.daemon.nimbus Nimbus$StandaloneINimbus]) (:import [org.apache.storm.generated StormTopology]) (:import [org.apache.storm.scheduler Cluster SupervisorDetails WorkerSlot ExecutorDetails SchedulerAssignmentImpl Topologies TopologyDetails])) @@ -127,7 +127,7 @@ assignment1 (SchedulerAssignmentImpl. "topology1" executor->slot1) assignment2 (SchedulerAssignmentImpl. "topology2" executor->slot2) assignment3 (SchedulerAssignmentImpl. "topology3" executor->slot3) - cluster (Cluster. (nimbus/standalone-nimbus) + cluster (Cluster. (Nimbus$StandaloneINimbus.) {"supervisor1" supervisor1 "supervisor2" supervisor2} {"topology1" assignment1 "topology2" assignment2 "topology3" assignment3} nil)] http://git-wip-us.apache.org/repos/asf/storm/blob/725003cc/storm-core/test/clj/org/apache/storm/security/auth/auth_test.clj ---------------------------------------------------------------------- diff --git a/storm-core/test/clj/org/apache/storm/security/auth/auth_test.clj b/storm-core/test/clj/org/apache/storm/security/auth/auth_test.clj index 56367e8..857ae6a 100644 --- a/storm-core/test/clj/org/apache/storm/security/auth/auth_test.clj +++ b/storm-core/test/clj/org/apache/storm/security/auth/auth_test.clj @@ -15,19 +15,25 @@ ;; limitations under the License. (ns org.apache.storm.security.auth.auth-test (:use [clojure test]) - (:require [org.apache.storm.daemon [nimbus :as nimbus]]) (:import [org.apache.thrift TException] [org.json.simple JSONValue] [org.apache.storm.utils Utils] [org.apache.storm.security.auth.authorizer ImpersonationAuthorizer] [java.net Inet4Address]) + (:import [org.apache.storm.blobstore BlobStore]) (:import [org.apache.thrift.transport TTransportException]) + (:import [org.apache.storm.testing.staticmocking MockedZookeeper]) + (:import [org.apache.storm.nimbus ILeaderElector]) + (:import [org.apache.storm.cluster IStormClusterState]) + (:import [org.mockito Mockito]) + (:import [org.apache.storm.zookeeper Zookeeper]) (:import [java.nio ByteBuffer]) (:import [java.security Principal AccessController]) (:import [javax.security.auth Subject]) (:import [java.net InetAddress]) (:import [org.apache.storm Config]) (:import [org.apache.storm.generated AuthorizationException]) + (:import [org.apache.storm.daemon.nimbus Nimbus$StandaloneINimbus]) (:import [org.apache.storm.utils NimbusClient ConfigUtils]) (:import [org.apache.storm.security.auth.authorizer SimpleWhitelistAuthorizer SimpleACLAuthorizer]) (:import [org.apache.storm.security.auth AuthUtils ThriftServer ThriftClient ShellBasedGroupsMapping @@ -57,21 +63,9 @@ (def nimbus-timeout (Integer. (* 3 1000))) (defn nimbus-data [storm-conf inimbus] - (let [forced-scheduler (.getForcedScheduler inimbus)] - {:conf storm-conf - :inimbus inimbus - :authorization-handler (StormCommon/mkAuthorizationHandler (storm-conf NIMBUS-AUTHORIZER) storm-conf) - :submitted-count (atom 0) - :storm-cluster-state nil - :submit-lock (Object.) - :heartbeats-cache (atom {}) - :downloaders nil - :uploaders nil - :uptime (Utils/makeUptimeComputer) - :validator nil - :timer nil - :scheduler nil - })) + (with-open [_ (MockedZookeeper. (proxy [Zookeeper] [] + (zkLeaderElectorImpl [conf blob-store] (Mockito/mock ILeaderElector))))] + (org.apache.storm.daemon.nimbus.Nimbus. storm-conf inimbus (Mockito/mock IStormClusterState) nil (Mockito/mock BlobStore) nil nil))) (defn dummy-service-handler ([conf inimbus auth-context] @@ -81,25 +75,25 @@ (^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) (if serializedConf (clojurify-structure (JSONValue/parse serializedConf))))) - (nimbus/check-authorization! nimbus-d storm-name @topo-conf "submitTopology" auth-context)) + (.checkAuthorization 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)) + (.checkAuthorization 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)) + (.checkAuthorization 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)) + (.checkAuthorization 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)) + (.checkAuthorization 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)) + (.checkAuthorization nimbus-d storm-name @topo-conf "deactivate" auth-context)) (uploadNewCredentials [this storm-name creds] - (nimbus/check-authorization! nimbus-d storm-name @topo-conf "uploadNewCredentials" auth-context)) + (.checkAuthorization nimbus-d storm-name @topo-conf "uploadNewCredentials" auth-context)) (beginFileUpload [this]) @@ -108,7 +102,7 @@ (^void finishFileUpload [this ^String location]) (^String beginFileDownload [this ^String file] - (nimbus/check-authorization! nimbus-d nil nil "fileDownload" auth-context) + (.checkAuthorization nimbus-d nil nil "fileDownload" auth-context) "Done!") (^ByteBuffer downloadChunk [this ^String id]) @@ -135,7 +129,7 @@ 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) + nimbus (Nimbus$StandaloneINimbus.) service-handler (dummy-service-handler conf nimbus) server (ThriftServer. conf http://git-wip-us.apache.org/repos/asf/storm/blob/725003cc/storm-core/test/clj/org/apache/storm/security/auth/nimbus_auth_test.clj ---------------------------------------------------------------------- diff --git a/storm-core/test/clj/org/apache/storm/security/auth/nimbus_auth_test.clj b/storm-core/test/clj/org/apache/storm/security/auth/nimbus_auth_test.clj index e13f2b5..2117b07 100644 --- a/storm-core/test/clj/org/apache/storm/security/auth/nimbus_auth_test.clj +++ b/storm-core/test/clj/org/apache/storm/security/auth/nimbus_auth_test.clj @@ -16,19 +16,21 @@ (ns org.apache.storm.security.auth.nimbus-auth-test (:use [clojure test]) (:require [org.apache.storm [testing :as testing]]) - (:require [org.apache.storm.daemon [nimbus :as nimbus]]) (:require [org.apache.storm.security.auth [auth-test :refer [nimbus-timeout]]]) (:import [java.nio ByteBuffer]) + (:import [java.util Optional]) (:import [org.apache.storm Config]) + (:import [org.apache.storm.blobstore BlobStore]) (:import [org.apache.storm.utils NimbusClient ConfigUtils]) - (:import [org.apache.storm.generated NotAliveException]) + (:import [org.apache.storm.generated NotAliveException StormBase]) (:import [org.apache.storm.security.auth AuthUtils ThriftServer ThriftClient ReqContext ThriftConnectionType]) (:import [org.apache.storm.generated Nimbus Nimbus$Client Nimbus$Processor AuthorizationException SubmitOptions TopologyInitialStatus KillOptions]) (:import [org.apache.storm.utils Utils]) - (:use [org.apache.storm util config log]) - (:use [org.apache.storm.daemon common nimbus]) + (:import [org.apache.storm.cluster IStormClusterState]) + (:import [org.mockito Mockito Matchers]) + (:use [org.apache.storm util config log testing]) (:require [conjure.core]) (:use [conjure core])) @@ -69,10 +71,16 @@ (.close client))))) (deftest test-noop-authorization-w-simple-transport - (let [port (Utils/getAvailablePort)] - (with-test-cluster [port nil - "org.apache.storm.security.auth.authorizer.NoopAuthorizer" - "org.apache.storm.security.auth.SimpleTransportPlugin"] + (let [port (Utils/getAvailablePort) + cluster-state (Mockito/mock IStormClusterState) + blob-store (Mockito/mock BlobStore) + topo-name "topo-name"] + (.thenReturn (Mockito/when (.getTopoId cluster-state topo-name)) (Optional/empty)) + (with-mocked-nimbus [cluster :cluster-state cluster-state :blob-store blob-store + :nimbus-daemon true + :daemon-conf {NIMBUS-AUTHORIZER "org.apache.storm.security.auth.authorizer.NoopAuthorizer" + NIMBUS-THRIFT-PORT port + STORM-THRIFT-TRANSPORT-PLUGIN "org.apache.storm.security.auth.SimpleTransportPlugin"}] (let [storm-conf (merge (clojurify-structure (ConfigUtils/readStormConfig)) {STORM-THRIFT-TRANSPORT-PLUGIN "org.apache.storm.security.auth.SimpleTransportPlugin" STORM-NIMBUS-RETRY-TIMES 0}) @@ -80,14 +88,22 @@ nimbus_client (.getClient client)] (testing "(Positive authorization) Authorization plugin should accept client request" (is (thrown-cause? NotAliveException - (.activate nimbus_client "topo-name")))) + (.activate nimbus_client topo-name)))) (.close client))))) (deftest test-deny-authorization-w-simple-transport - (let [port (Utils/getAvailablePort)] - (with-test-cluster [port nil - "org.apache.storm.security.auth.authorizer.DenyAuthorizer" - "org.apache.storm.security.auth.SimpleTransportPlugin"] + (let [port (Utils/getAvailablePort) + cluster-state (Mockito/mock IStormClusterState) + blob-store (Mockito/mock BlobStore) + topo-name "topo-name" + topo-id "topo-name-1"] + (.thenReturn (Mockito/when (.getTopoId cluster-state topo-name)) (Optional/of topo-id)) + (.thenReturn (Mockito/when (.readTopologyConf blob-store (Mockito/any String) (Mockito/anyObject))) {}) + (with-mocked-nimbus [cluster :cluster-state cluster-state :blob-store blob-store + :nimbus-daemon true + :daemon-conf {NIMBUS-AUTHORIZER "org.apache.storm.security.auth.authorizer.DenyAuthorizer" + NIMBUS-THRIFT-PORT port + STORM-THRIFT-TRANSPORT-PLUGIN "org.apache.storm.security.auth.SimpleTransportPlugin"}] (let [storm-conf (merge (clojurify-structure (ConfigUtils/readStormConfig)) {STORM-THRIFT-TRANSPORT-PLUGIN "org.apache.storm.security.auth.SimpleTransportPlugin" Config/NIMBUS_THRIFT_PORT port @@ -96,8 +112,8 @@ 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 (.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))) @@ -106,19 +122,15 @@ (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")))) + (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))) + (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 @@ -140,13 +152,21 @@ (.close client))))) (deftest test-deny-authorization-w-sasl-digest - (let [port (Utils/getAvailablePort)] - (with-test-cluster [port - "test/clj/org/apache/storm/security/auth/jaas_digest.conf" - "org.apache.storm.security.auth.authorizer.DenyAuthorizer" - "org.apache.storm.security.auth.digest.DigestSaslTransportPlugin"] + (let [port (Utils/getAvailablePort) + cluster-state (Mockito/mock IStormClusterState) + blob-store (Mockito/mock BlobStore) + topo-name "topo-name" + topo-id "topo-name-1"] + (.thenReturn (Mockito/when (.getTopoId cluster-state topo-name)) (Optional/of topo-id)) + (.thenReturn (Mockito/when (.readTopologyConf blob-store (Mockito/any String) (Mockito/anyObject))) {}) + (with-mocked-nimbus [cluster :cluster-state cluster-state :blob-store blob-store + :nimbus-daemon true + :daemon-conf {NIMBUS-AUTHORIZER "org.apache.storm.security.auth.authorizer.DenyAuthorizer" + "java.security.auth.login.config" "test/clj/org/apache/storm/security/auth/jaas_digest.conf" + NIMBUS-THRIFT-PORT port + STORM-THRIFT-TRANSPORT-PLUGIN "org.apache.storm.security.auth.digest.DigestSaslTransportPlugin"}] (let [storm-conf (merge (clojurify-structure (ConfigUtils/readStormConfig)) - {STORM-THRIFT-TRANSPORT-PLUGIN "org.apache.storm.security.auth.digest.DigestSaslTransportPlugin" + {STORM-THRIFT-TRANSPORT-PLUGIN "org.apache.storm.security.auth.digest.DigestSaslTransportPlugin" "java.security.auth.login.config" "test/clj/org/apache/storm/security/auth/jaas_digest.conf" Config/NIMBUS_THRIFT_PORT port STORM-NIMBUS-RETRY-TIMES 0}) @@ -154,26 +174,24 @@ 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 (.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")))) + (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))) + (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)))))