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 2016/02/24 17:17:17 UTC

[07/27] storm git commit: update class hierarchy about cluster

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/test/clj/org/apache/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/nimbus_test.clj b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
index d4402fb..772a232 100644
--- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj
+++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
@@ -36,7 +36,7 @@
   (:import [org.apache.storm.utils Time Utils ConfigUtils])
   (:import [org.apache.storm.zookeeper Zookeeper])
   (:import [org.apache.commons.io FileUtils])
-  (:import [org.apache.storm.cluster StormZkClusterState ClusterStateContext Cluster])
+  (:import [org.apache.storm.cluster StormClusterStateImpl ClusterStateContext ClusterUtils])
   (:use [org.apache.storm testing MockAutoCred util config log timer converter])
   (:use [org.apache.storm.daemon common])
   (:require [conjure.core])
@@ -126,7 +126,7 @@
   (let [state (:storm-cluster-state cluster)
         executor->node+port (:executor->node+port (clojurify-assignment (.assignmentInfo state storm-id nil)))
         [node port] (get executor->node+port executor)
-        curr-beat (clojurify-zk-worker-hb (.getworkerHeartbeat state storm-id node port))
+        curr-beat (clojurify-zk-worker-hb (.getWorkerHeartbeat state storm-id node port))
         stats (:executor-stats curr-beat)]
     (.workerHeartbeat state storm-id node port
       (thriftify-zk-worker-hb {:storm-id storm-id :time-secs (current-time-secs) :uptime 10 :executor-stats (merge stats {executor (stats/render-stats! (stats/mk-bolt-stats 20))})})
@@ -1031,7 +1031,7 @@
                             STORM-CLUSTER-MODE "local"
                             STORM-ZOOKEEPER-PORT zk-port
                             STORM-LOCAL-DIR nimbus-dir}))
-         (bind cluster-state (StormZkClusterState. conf nil (ClusterStateContext.)))
+         (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
          (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
          (bind topology (thrift/mk-topology
                          {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
@@ -1103,7 +1103,7 @@
                         STORM-CLUSTER-MODE "local"
                         STORM-ZOOKEEPER-PORT zk-port
                         STORM-LOCAL-DIR nimbus-dir}))
-          (bind cluster-state (StormZkClusterState. conf nil (ClusterStateContext.)))
+          (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
           (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
           (bind topology (thrift/mk-topology
                            {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
@@ -1113,7 +1113,7 @@
                           (zkLeaderElectorImpl [conf] (mock-leader-elector :is-leader false))))]
 
             (letlocals
-              (bind non-leader-cluster-state (StormZkClusterState. conf nil (ClusterStateContext.)))
+              (bind non-leader-cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
               (bind non-leader-nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
 
               ;first we verify that the master nimbus can perform all actions, even with another nimbus present.
@@ -1309,7 +1309,7 @@
                         :status {:type bogus-type}}
                 }
         ]
-      (stubbing [topology-bases bogus-bases
+      (stubbing [nimbus/nimbus-topology-bases bogus-bases
                  nimbus/get-blob-replication-count 1]
         (let [topos (.get_topologies (.getClusterInfo nimbus))]
           ; The number of topologies in the summary is correct.
@@ -1350,14 +1350,13 @@
                      NIMBUS-THRIFT-PORT 6666})
           expected-acls nimbus/NIMBUS-ZK-ACLS
           fake-inimbus (reify INimbus (getForcedScheduler [this] nil))
-          storm-zk (Mockito/mock Cluster)]
+          cluster-utils (Mockito/mock ClusterUtils)]
       (with-open [_ (proxy [MockedConfigUtils] []
                       (nimbusTopoHistoryStateImpl [conf] nil))
                   zk-le (MockedZookeeper. (proxy [Zookeeper] []
                           (zkLeaderElectorImpl [conf] nil)))
-                  storm-zk-le (MockedCluster. storm-zk)]
+                  mocked-cluster (MockedCluster. cluster-utils)]
         (stubbing [mk-authorization-handler nil
-               ;  cluster/mk-storm-cluster-state nil
                  nimbus/file-cache-map nil
                  nimbus/mk-blob-cache-map nil
                  nimbus/mk-bloblist-cache-map nil
@@ -1366,10 +1365,7 @@
                  mk-timer nil
                  nimbus/mk-scheduler nil]
           (nimbus/nimbus-data auth-conf fake-inimbus)
-          (.mkStormClusterStateImpl (Mockito/verify storm-zk (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
-          ;   (verify-call-times-for cluster/mk-storm-cluster-state 1)
-       ; (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
-       ; expected-acls)
+          (.mkStormClusterStateImpl (Mockito/verify cluster-utils (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
           )))))
 
 (deftest test-file-bogus-download
@@ -1401,7 +1397,7 @@
                       STORM-CLUSTER-MODE "local"
                       STORM-ZOOKEEPER-PORT zk-port
                       STORM-LOCAL-DIR nimbus-dir}))
-        (bind cluster-state (StormZkClusterState. conf nil (ClusterStateContext.)))
+        (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
         (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
         (sleep-secs 1)
         (bind topology (thrift/mk-topology
@@ -1433,7 +1429,7 @@
                         STORM-ZOOKEEPER-PORT zk-port
                         STORM-LOCAL-DIR nimbus-dir
                         NIMBUS-TOPOLOGY-ACTION-NOTIFIER-PLUGIN (.getName InMemoryTopologyActionNotifier)}))
-          (bind cluster-state (StormZkClusterState. conf nil (ClusterStateContext.)))
+          (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
           (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
           (bind notifier (InMemoryTopologyActionNotifier.))
           (sleep-secs 1)

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/test/clj/org/apache/storm/supervisor_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/supervisor_test.clj b/storm-core/test/clj/org/apache/storm/supervisor_test.clj
index c98a68b..b89b7bb 100644
--- a/storm-core/test/clj/org/apache/storm/supervisor_test.clj
+++ b/storm-core/test/clj/org/apache/storm/supervisor_test.clj
@@ -29,7 +29,7 @@
   (:import [org.mockito.exceptions.base MockitoAssertionError])
   (:import [java.io File])
   (:import [java.nio.file Files])
-  (:import [org.apache.storm.cluster StormZkClusterState Cluster ClusterStateContext])
+  (:import [org.apache.storm.cluster StormClusterStateImpl ClusterStateContext ClusterUtils])
   (:import [java.nio.file.attribute FileAttribute])
   (:use [org.apache.storm config testing util timer log converter])
   (:use [org.apache.storm.daemon common])
@@ -565,17 +565,17 @@
           fake-isupervisor (reify ISupervisor
                              (getSupervisorId [this] nil)
                              (getAssignmentId [this] nil))
-          storm-zk (Mockito/mock Cluster)]
+          cluster-utils (Mockito/mock ClusterUtils)]
       (with-open [_ (proxy [MockedConfigUtils] []
                       (supervisorStateImpl [conf] nil)
                       (supervisorLocalDirImpl [conf] nil))
-                  storm-zk-le (MockedCluster. storm-zk)]
+                  mocked-cluster (MockedCluster. cluster-utils)]
         (stubbing [uptime-computer nil
               ;   cluster/mk-storm-cluster-state nil
                  local-hostname nil
                  mk-timer nil]
           (supervisor/supervisor-data auth-conf nil fake-isupervisor)
-          (.mkStormClusterStateImpl (Mockito/verify storm-zk (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
+          (.mkStormClusterStateImpl (Mockito/verify cluster-utils (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
         ;  (verify-call-times-for cluster/mk-storm-cluster-state 1)
         ;  (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
         ;                                     expected-acls)

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/test/jvm/org/apache/storm/ClusterTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/ClusterTest.java b/storm-core/test/jvm/org/apache/storm/ClusterTest.java
deleted file mode 100644
index ef43afe..0000000
--- a/storm-core/test/jvm/org/apache/storm/ClusterTest.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm;
-
-
-public class ClusterTest {
-}