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/23 21:19:56 UTC

[1/8] storm git commit: STORM-1255: port storm_utils.clj to java and split Time tests into its own test file

Repository: storm
Updated Branches:
  refs/heads/master 58050a5b3 -> 53446108b


STORM-1255: port storm_utils.clj to java and split Time tests into its
own test file


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

Branch: refs/heads/master
Commit: a2a656ed3fdbf76fddb730bced5bfe7f2b18df72
Parents: 4699990
Author: Alessandro Bellina <ab...@yahoo-inc.com>
Authored: Mon Feb 15 13:30:10 2016 -0600
Committer: Alessandro Bellina <ab...@yahoo-inc.com>
Committed: Tue Feb 16 23:17:42 2016 -0600

----------------------------------------------------------------------
 .../src/jvm/org/apache/storm/utils/Utils.java   |  17 +-
 .../test/clj/org/apache/storm/utils_test.clj    | 111 ----------
 .../jvm/org/apache/storm/utils/TimeTest.java    | 106 +++++++++
 .../jvm/org/apache/storm/utils/UtilsTest.java   | 221 +++++++++++++++++++
 4 files changed, 337 insertions(+), 118 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/a2a656ed/storm-core/src/jvm/org/apache/storm/utils/Utils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/utils/Utils.java b/storm-core/src/jvm/org/apache/storm/utils/Utils.java
index 9a849ea..5674459 100644
--- a/storm-core/src/jvm/org/apache/storm/utils/Utils.java
+++ b/storm-core/src/jvm/org/apache/storm/utils/Utils.java
@@ -1058,6 +1058,10 @@ public class Utils {
         return newCurator(conf, servers, port, root, null);
     }
 
+    public static CuratorFramework newCurator(Map conf, List<String> servers, Object port, ZookeeperAuthInfo auth) {
+        return newCurator(conf, servers, port, "", auth);
+    }
+
     public static CuratorFramework newCurator(Map conf, List<String> servers, Object port, String root, ZookeeperAuthInfo auth) {
         List<String> serverPorts = new ArrayList<String>();
         for (String zkServer : servers) {
@@ -1113,10 +1117,6 @@ public class Utils {
         setupBuilder(builder, zkStr, conf, auth);
     }
 
-    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, ZookeeperAuthInfo auth) {
         CuratorFramework ret = newCurator(conf, servers, port, root, auth);
         ret.start();
@@ -1397,13 +1397,16 @@ public class Utils {
             }
             if (memoryOpts != null) {
                 int unit = 1;
-                if (memoryOpts.toLowerCase().endsWith("k")) {
+                memoryOpts = memoryOpts.toLowerCase();
+
+                if (memoryOpts.endsWith("k")) {
                     unit = 1024;
-                } else if (memoryOpts.toLowerCase().endsWith("m")) {
+                } else if (memoryOpts.endsWith("m")) {
                     unit = 1024 * 1024;
-                } else if (memoryOpts.toLowerCase().endsWith("g")) {
+                } else if (memoryOpts.endsWith("g")) {
                     unit = 1024 * 1024 * 1024;
                 }
+
                 memoryOpts = memoryOpts.replaceAll("[a-zA-Z]", "");
                 Double result =  Double.parseDouble(memoryOpts) * unit / 1024.0 / 1024.0;
                 return (result < 1.0) ? 1.0 : result;

http://git-wip-us.apache.org/repos/asf/storm/blob/a2a656ed/storm-core/test/clj/org/apache/storm/utils_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/utils_test.clj b/storm-core/test/clj/org/apache/storm/utils_test.clj
deleted file mode 100644
index 26442aa..0000000
--- a/storm-core/test/clj/org/apache/storm/utils_test.clj
+++ /dev/null
@@ -1,111 +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.
-(ns org.apache.storm.utils-test
-  (:import [org.apache.storm Config])
-  (:import [org.apache.storm.utils NimbusClient Utils])
-  (:import [org.apache.curator.retry ExponentialBackoffRetry])
-  (:import [org.apache.thrift.transport TTransportException])
-  (:import [org.apache.storm.utils ConfigUtils Time])
-  (:use [org.apache.storm config util])
-  (:use [clojure test])
-)
-
-(deftest test-new-curator-uses-exponential-backoff
-  (let [expected_interval 2400
-        expected_retries 10
-        expected_ceiling 3000
-        conf (merge (clojurify-structure (Utils/readDefaultConfig))
-          {Config/STORM_ZOOKEEPER_RETRY_INTERVAL expected_interval
-           Config/STORM_ZOOKEEPER_RETRY_TIMES expected_retries
-           Config/STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING expected_ceiling})
-        servers ["bogus_server"]
-        arbitrary_port 42
-        curator (Utils/newCurator conf servers arbitrary_port nil)
-        retry (-> curator .getZookeeperClient .getRetryPolicy)
-       ]
-    (is (.isAssignableFrom ExponentialBackoffRetry (.getClass retry)))
-    (is (= (.getBaseSleepTimeMs retry) expected_interval))
-    (is (= (.getN retry) expected_retries))
-    (is (= (.getSleepTimeMs retry 10 0) expected_ceiling))
-  )
-)
-
-(deftest test-getConfiguredClient-throws-RunTimeException-on-bad-args
-  (let [storm-conf (merge
-                     (clojurify-structure (ConfigUtils/readStormConfig))
-                    {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 (Time/secsToMillisLong 0)))
-  (is (= 2 (Time/secsToMillisLong 0.002)))
-  (is (= 500 (Time/secsToMillisLong 0.5)))
-  (is (= 1000 (Time/secsToMillisLong 1)))
-  (is (= 1080 (Time/secsToMillisLong 1.08)))
-  (is (= 10000 (Time/secsToMillisLong 10)))
-  (is (= 10100 (Time/secsToMillisLong 10.1)))
-)
-

http://git-wip-us.apache.org/repos/asf/storm/blob/a2a656ed/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java b/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java
new file mode 100644
index 0000000..faf75eb
--- /dev/null
+++ b/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java
@@ -0,0 +1,106 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.utils;
+
+import org.junit.Test;
+import org.junit.Assert;
+
+public class TimeTest{
+
+    @Test
+    public void secsToMillisLongTest(){
+        Assert.assertEquals(Time.secsToMillisLong(0),     0);
+        Assert.assertEquals(Time.secsToMillisLong(0.002), 2);
+        Assert.assertEquals(Time.secsToMillisLong(1),     1000);
+        Assert.assertEquals(Time.secsToMillisLong(1.08),  1080);
+        Assert.assertEquals(Time.secsToMillisLong(10),    10000);
+        Assert.assertEquals(Time.secsToMillisLong(10.1),  10100);
+    }
+
+    @Test
+    public void ifNotSimulatingIsSimulatingReturnsFalse(){
+        Assert.assertFalse(Time.isSimulating());
+    }
+
+    @Test
+    public void ifSimulatingIsSimulatingReturnsTrue(){
+        Time.startSimulating();
+        Assert.assertTrue(Time.isSimulating());
+        Time.stopSimulating();
+    }
+
+    @Test
+    public void advanceTimeSimulatedTimeBy0Causes0DeltaTest(){
+        Time.startSimulating();
+        long current = Time.currentTimeMillis();
+        Time.advanceTime(0);
+        Assert.assertEquals(Time.deltaMs(current), 0);
+        Time.stopSimulating();
+    }
+
+    @Test
+    public void advanceTimeSimulatedTimeBy1000Causes1000MsDeltaTest(){
+        Time.startSimulating();
+        long current = Time.currentTimeMillis();
+        Time.advanceTime(1000);
+        Assert.assertEquals(Time.deltaMs(current), 1000);
+        Time.stopSimulating();
+    }
+
+    @Test
+    public void advanceTimeSimulatedTimeBy1500Causes1500MsDeltaTest(){
+        Time.startSimulating();
+        long current = Time.currentTimeMillis();
+        Time.advanceTime(1500);
+        Assert.assertEquals(Time.deltaMs(current), 1500);
+        Time.stopSimulating();
+    }
+
+    @Test
+    public void advanceTimeSimulatedTimeByNegative1500CausesNegative1500MsDeltaTest(){
+        Time.startSimulating();
+        long current = Time.currentTimeMillis();
+        Time.advanceTime(-1500);
+        Assert.assertEquals(Time.deltaMs(current), -1500);
+        Time.stopSimulating();
+    }
+
+    @Test
+    public void advanceSimulatedTimeBy1000MsSecondReturns1SecondTest(){
+        Time.startSimulating();
+        int current = Time.currentTimeSecs();
+        Time.advanceTime(1000);
+        Assert.assertEquals(Time.deltaSecs(current), 1);
+        Time.stopSimulating();
+    }
+
+    @Test
+    public void advanceSimulatedtimeBy1500MsSecondsReturns1TruncatedSecondTest(){
+        Time.startSimulating();
+        int current = Time.currentTimeSecs();
+        Time.advanceTime(1500);
+        Assert.assertEquals(Time.deltaSecs(current), 1, 0);
+        Time.stopSimulating();
+    }
+
+    @Test(expected=IllegalStateException.class)
+    public void ifNotSimulatingAdvanceTimeThrows(){
+        Time.advanceTime(1000);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/a2a656ed/storm-core/test/jvm/org/apache/storm/utils/UtilsTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/utils/UtilsTest.java b/storm-core/test/jvm/org/apache/storm/utils/UtilsTest.java
new file mode 100644
index 0000000..1bb5f71
--- /dev/null
+++ b/storm-core/test/jvm/org/apache/storm/utils/UtilsTest.java
@@ -0,0 +1,221 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.utils;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.HashMap;
+import org.junit.Test;
+import org.junit.Assert;
+
+import org.apache.curator.ensemble.exhibitor.ExhibitorEnsembleProvider;
+import org.apache.curator.ensemble.fixed.FixedEnsembleProvider;
+import org.apache.curator.framework.AuthInfo;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+
+import org.apache.storm.Config;
+import org.apache.thrift.transport.TTransportException;
+
+import static org.mockito.Mockito.*;
+
+public class UtilsTest{
+    @Test
+    public void newCuratorUsesExponentialBackoffTest() throws InterruptedException{
+        final int expectedInterval = 2400;
+        final int expectedRetries = 10;
+        final int expectedCeiling = 3000;
+
+        Map<String, Object> config = Utils.readDefaultConfig();
+        config.put(Config.STORM_ZOOKEEPER_RETRY_INTERVAL, expectedInterval); 
+        config.put(Config.STORM_ZOOKEEPER_RETRY_TIMES, expectedRetries); 
+        config.put(Config.STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING, expectedCeiling); 
+
+        CuratorFramework curator = Utils.newCurator(config, Arrays.asList("bogus_server"), 42 /*port*/, "");
+        StormBoundedExponentialBackoffRetry policy = 
+            (StormBoundedExponentialBackoffRetry) curator.getZookeeperClient().getRetryPolicy();
+        Assert.assertEquals(policy.getBaseSleepTimeMs(), expectedInterval);
+        Assert.assertEquals(policy.getN(), expectedRetries);
+        Assert.assertEquals(policy.getSleepTimeMs(10, 0), expectedCeiling);
+    }
+
+    @Test(expected = RuntimeException.class)
+    public void getConfiguredClientThrowsRuntimeExceptionOnBadArgsTest () throws RuntimeException, TTransportException {
+        Map config = ConfigUtils.readStormConfig();
+        config.put(Config.STORM_NIMBUS_RETRY_TIMES, 0);
+        new NimbusClient(config, "", 65535);
+    }
+
+    private Map mockMap(String key, String value){
+        Map<String, Object> map = new HashMap<String, Object>();
+        map.put(key, value);
+        return map;
+    }
+
+    private Map topologyMockMap(String value){
+        return mockMap(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME, value);
+    }
+
+    private Map serverMockMap(String value){
+        return mockMap(Config.STORM_ZOOKEEPER_AUTH_SCHEME, value);
+    }
+
+    private Map emptyMockMap(){
+        return new HashMap<String, Object>();
+    }
+
+    /* isZkAuthenticationConfiguredTopology */
+    @Test
+    public void isZkAuthenticationConfiguredTopologyReturnsFalseOnNullConfigTest(){
+        Assert.assertFalse(Utils.isZkAuthenticationConfiguredTopology(null));
+    }
+
+    @Test
+    public void isZkAuthenticationConfiguredTopologyReturnsFalseOnSchemeKeyMissingTest(){
+        Assert.assertFalse(Utils.isZkAuthenticationConfiguredTopology(emptyMockMap()));
+    }
+
+    @Test
+    public void isZkAuthenticationConfiguredTopologyReturnsFalseOnSchemeValueNullTest(){
+        Assert.assertFalse(Utils.isZkAuthenticationConfiguredTopology(topologyMockMap(null)));
+    }
+
+    @Test
+    public void isZkAuthenticationConfiguredTopologyReturnsTrueWhenSchemeSetToStringTest(){
+        Assert.assertTrue(Utils.isZkAuthenticationConfiguredTopology(topologyMockMap("foobar")));
+    }
+
+    /* isZkAuthenticationConfiguredStormServer */
+    @Test
+    public void isZkAuthenticationConfiguredStormReturnsFalseOnNullConfigTest(){
+        Assert.assertFalse(Utils.isZkAuthenticationConfiguredStormServer(null));
+    }
+
+    @Test
+    public void isZkAuthenticationConfiguredStormReturnsFalseOnSchemeKeyMissingTest(){
+        Assert.assertFalse(Utils.isZkAuthenticationConfiguredStormServer(emptyMockMap()));
+    }
+
+    @Test
+    public void isZkAuthenticationConfiguredStormReturnsFalseOnSchemeValueNullTest(){
+        Assert.assertFalse(Utils.isZkAuthenticationConfiguredStormServer(serverMockMap(null)));
+    }
+
+    @Test
+    public void isZkAuthenticationConfiguredStormReturnsTrueWhenSchemeSetToStringTest(){
+        Assert.assertTrue(Utils.isZkAuthenticationConfiguredStormServer(serverMockMap("foobar")));
+    }
+
+    @Test
+    public void isZkAuthenticationConfiguredStormReturnsTrueWhenAuthLoginConfigIsSetTest(){
+        String key = "java.security.auth.login.config";
+        String oldValue = System.getProperty(key);
+        try {
+            System.setProperty("java.security.auth.login.config", "anything");
+            Assert.assertTrue(Utils.isZkAuthenticationConfiguredStormServer(emptyMockMap()));
+        } catch (Exception ignore) {
+        } finally {
+            // reset property
+            if (oldValue == null){
+                System.clearProperty(key);
+            } else {
+                System.setProperty(key, oldValue);
+            }
+        }
+    }
+
+    private CuratorFrameworkFactory.Builder setupBuilder(boolean withExhibitor){
+        return setupBuilder(withExhibitor, false /*without Auth*/);
+    }
+
+    private CuratorFrameworkFactory.Builder setupBuilder(boolean withExhibitor, boolean withAuth){
+        CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder();
+        Map<String, Object> conf = new HashMap<String, Object>();
+        if (withExhibitor){
+            conf.put(Config.STORM_EXHIBITOR_SERVERS,"foo");
+            conf.put(Config.STORM_EXHIBITOR_PORT, 0);
+            conf.put(Config.STORM_EXHIBITOR_URIPATH, "/exhibitor");
+            conf.put(Config.STORM_EXHIBITOR_POLL, 0);
+            conf.put(Config.STORM_EXHIBITOR_RETRY_INTERVAL, 0);
+            conf.put(Config.STORM_EXHIBITOR_RETRY_INTERVAL_CEILING, 0);
+            conf.put(Config.STORM_EXHIBITOR_RETRY_TIMES, 0);
+        }
+        conf.put(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT, 0);
+        conf.put(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT, 0);
+        conf.put(Config.STORM_ZOOKEEPER_RETRY_INTERVAL, 0);
+        conf.put(Config.STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING, 0);
+        conf.put(Config.STORM_ZOOKEEPER_RETRY_TIMES, 0);
+        String zkStr = new String("zk_connection_string");
+        ZookeeperAuthInfo auth = null;
+        if (withAuth){
+            auth = new ZookeeperAuthInfo("scheme", "abc".getBytes());
+        }
+        Utils.testSetupBuilder(builder, zkStr, conf, auth);
+        return builder;
+    }
+
+    @Test
+    public void ifExhibitorServersProvidedBuilderUsesTheExhibitorEnsembleProviderTest(){
+        CuratorFrameworkFactory.Builder builder = setupBuilder(true /*with exhibitor*/);
+        Assert.assertEquals(builder.getEnsembleProvider().getConnectionString(), "");
+        Assert.assertEquals(builder.getEnsembleProvider().getClass(), ExhibitorEnsembleProvider.class);
+    }
+
+    @Test
+    public void ifExhibitorServersAreEmptyBuilderUsesAFixedEnsembleProviderTest(){
+        CuratorFrameworkFactory.Builder builder = setupBuilder(false /*without exhibitor*/);
+        Assert.assertEquals(builder.getEnsembleProvider().getConnectionString(), "zk_connection_string");
+        Assert.assertEquals(builder.getEnsembleProvider().getClass(), FixedEnsembleProvider.class);
+    }
+
+    @Test
+    public void ifAuthSchemeAndPayloadAreDefinedBuilderUsesAuthTest(){
+        CuratorFrameworkFactory.Builder builder = setupBuilder(false /*without exhibitor*/, true /*with auth*/);
+        List<AuthInfo> authInfos = builder.getAuthInfos();
+        AuthInfo authInfo = authInfos.get(0);
+        Assert.assertEquals(authInfo.getScheme(), "scheme"); 
+        Assert.assertArrayEquals(authInfo.getAuth(), "abc".getBytes());
+    }
+
+    @Test
+    public void parseJvmHeapMemByChildOpts1024KIs1Test(){
+        Assert.assertEquals(Utils.parseJvmHeapMemByChildOpts("Xmx1024K", 0.0).doubleValue(), 1.0, 0); 
+    }
+
+    @Test
+    public void parseJvmHeapMemByChildOpts100MIs100Test(){
+        Assert.assertEquals(Utils.parseJvmHeapMemByChildOpts("Xmx100M", 0.0).doubleValue(), 100.0, 0); 
+    }
+
+    @Test
+    public void parseJvmHeapMemByChildOpts1GIs1024Test(){
+        Assert.assertEquals(Utils.parseJvmHeapMemByChildOpts("Xmx1G", 0.0).doubleValue(), 1024.0, 0); 
+    }
+
+    @Test
+    public void parseJvmHeapMemByChildOptsReturnsDefaultIfMatchNotFoundTest(){
+        Assert.assertEquals(Utils.parseJvmHeapMemByChildOpts("Xmx1T", 123.0).doubleValue(), 123.0, 0); 
+    }
+
+    @Test
+    public void parseJvmHeapMemByChildOptsReturnsDefaultIfInputIsNullTest(){
+        Assert.assertEquals(Utils.parseJvmHeapMemByChildOpts(null, 123.0).doubleValue(), 123.0, 0); 
+    }
+}


[2/8] storm git commit: STORM-1255: remove extra import, adjust formatting, shorten function names by grouping assertions

Posted by bo...@apache.org.
STORM-1255: remove extra import, adjust formatting, shorten function names by grouping assertions


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

Branch: refs/heads/master
Commit: 20851f8b2d47bc74fa8ac36c78c43e038d56ed81
Parents: a2a656e
Author: Alessandro Bellina <ab...@yahoo-inc.com>
Authored: Wed Feb 17 12:26:02 2016 -0600
Committer: Alessandro Bellina <ab...@yahoo-inc.com>
Committed: Wed Feb 17 12:26:02 2016 -0600

----------------------------------------------------------------------
 .../jvm/org/apache/storm/utils/TimeTest.java    |  40 +++---
 .../jvm/org/apache/storm/utils/UtilsTest.java   | 122 +++++++++----------
 2 files changed, 75 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/20851f8b/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java b/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java
index faf75eb..eb5e1d5 100644
--- a/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java
+++ b/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java
@@ -21,10 +21,10 @@ package org.apache.storm.utils;
 import org.junit.Test;
 import org.junit.Assert;
 
-public class TimeTest{
+public class TimeTest {
 
     @Test
-    public void secsToMillisLongTest(){
+    public void secsToMillisLongTest() {
         Assert.assertEquals(Time.secsToMillisLong(0),     0);
         Assert.assertEquals(Time.secsToMillisLong(0.002), 2);
         Assert.assertEquals(Time.secsToMillisLong(1),     1000);
@@ -34,19 +34,24 @@ public class TimeTest{
     }
 
     @Test
-    public void ifNotSimulatingIsSimulatingReturnsFalse(){
+    public void ifNotSimulatingIsSimulatingReturnsFalse() {
         Assert.assertFalse(Time.isSimulating());
     }
 
+    @Test(expected=IllegalStateException.class)
+    public void ifNotSimulatingAdvanceTimeThrows() {
+        Time.advanceTime(1000);
+    }
+
     @Test
-    public void ifSimulatingIsSimulatingReturnsTrue(){
+    public void ifSimulatingIsSimulatingReturnsTrue() {
         Time.startSimulating();
         Assert.assertTrue(Time.isSimulating());
         Time.stopSimulating();
     }
 
     @Test
-    public void advanceTimeSimulatedTimeBy0Causes0DeltaTest(){
+    public void shouldNotAdvanceTimeTest() {
         Time.startSimulating();
         long current = Time.currentTimeMillis();
         Time.advanceTime(0);
@@ -55,34 +60,29 @@ public class TimeTest{
     }
 
     @Test
-    public void advanceTimeSimulatedTimeBy1000Causes1000MsDeltaTest(){
+    public void shouldAdvanceForwardTest() {
         Time.startSimulating();
         long current = Time.currentTimeMillis();
         Time.advanceTime(1000);
         Assert.assertEquals(Time.deltaMs(current), 1000);
-        Time.stopSimulating();
-    }
-
-    @Test
-    public void advanceTimeSimulatedTimeBy1500Causes1500MsDeltaTest(){
-        Time.startSimulating();
-        long current = Time.currentTimeMillis();
-        Time.advanceTime(1500);
+        Time.advanceTime(500);
         Assert.assertEquals(Time.deltaMs(current), 1500);
         Time.stopSimulating();
     }
 
     @Test
-    public void advanceTimeSimulatedTimeByNegative1500CausesNegative1500MsDeltaTest(){
+    public void shouldAdvanceBackwardsTest() {
         Time.startSimulating();
         long current = Time.currentTimeMillis();
+        Time.advanceTime(1000);
+        Assert.assertEquals(Time.deltaMs(current), 1000);
         Time.advanceTime(-1500);
-        Assert.assertEquals(Time.deltaMs(current), -1500);
+        Assert.assertEquals(Time.deltaMs(current), -500);
         Time.stopSimulating();
     }
 
     @Test
-    public void advanceSimulatedTimeBy1000MsSecondReturns1SecondTest(){
+    public void deltaSecsConvertsToSecondsTest() {
         Time.startSimulating();
         int current = Time.currentTimeSecs();
         Time.advanceTime(1000);
@@ -91,7 +91,7 @@ public class TimeTest{
     }
 
     @Test
-    public void advanceSimulatedtimeBy1500MsSecondsReturns1TruncatedSecondTest(){
+    public void deltaSecsTruncatesFractionalSeconds() {
         Time.startSimulating();
         int current = Time.currentTimeSecs();
         Time.advanceTime(1500);
@@ -99,8 +99,4 @@ public class TimeTest{
         Time.stopSimulating();
     }
 
-    @Test(expected=IllegalStateException.class)
-    public void ifNotSimulatingAdvanceTimeThrows(){
-        Time.advanceTime(1000);
-    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/20851f8b/storm-core/test/jvm/org/apache/storm/utils/UtilsTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/utils/UtilsTest.java b/storm-core/test/jvm/org/apache/storm/utils/UtilsTest.java
index 1bb5f71..8583a16 100644
--- a/storm-core/test/jvm/org/apache/storm/utils/UtilsTest.java
+++ b/storm-core/test/jvm/org/apache/storm/utils/UtilsTest.java
@@ -34,11 +34,9 @@ import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.storm.Config;
 import org.apache.thrift.transport.TTransportException;
 
-import static org.mockito.Mockito.*;
-
-public class UtilsTest{
+public class UtilsTest {
     @Test
-    public void newCuratorUsesExponentialBackoffTest() throws InterruptedException{
+    public void newCuratorUsesExponentialBackoffTest() throws InterruptedException {
         final int expectedInterval = 2400;
         final int expectedRetries = 10;
         final int expectedCeiling = 3000;
@@ -63,68 +61,64 @@ public class UtilsTest{
         new NimbusClient(config, "", 65535);
     }
 
-    private Map mockMap(String key, String value){
+    private Map mockMap(String key, String value) {
         Map<String, Object> map = new HashMap<String, Object>();
         map.put(key, value);
         return map;
     }
 
-    private Map topologyMockMap(String value){
+    private Map topologyMockMap(String value) {
         return mockMap(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME, value);
     }
 
-    private Map serverMockMap(String value){
+    private Map serverMockMap(String value) {
         return mockMap(Config.STORM_ZOOKEEPER_AUTH_SCHEME, value);
     }
 
-    private Map emptyMockMap(){
+    private Map emptyMockMap() {
         return new HashMap<String, Object>();
     }
 
-    /* isZkAuthenticationConfiguredTopology */
     @Test
-    public void isZkAuthenticationConfiguredTopologyReturnsFalseOnNullConfigTest(){
-        Assert.assertFalse(Utils.isZkAuthenticationConfiguredTopology(null));
-    }
+    public void isZkAuthenticationConfiguredTopologyTest() {
+        Assert.assertFalse(
+            "Returns null if given null config", 
+            Utils.isZkAuthenticationConfiguredTopology(null));
 
-    @Test
-    public void isZkAuthenticationConfiguredTopologyReturnsFalseOnSchemeKeyMissingTest(){
-        Assert.assertFalse(Utils.isZkAuthenticationConfiguredTopology(emptyMockMap()));
-    }
+        Assert.assertFalse(
+            "Returns false if scheme key is missing", 
+            Utils.isZkAuthenticationConfiguredTopology(emptyMockMap()));
 
-    @Test
-    public void isZkAuthenticationConfiguredTopologyReturnsFalseOnSchemeValueNullTest(){
-        Assert.assertFalse(Utils.isZkAuthenticationConfiguredTopology(topologyMockMap(null)));
-    }
+        Assert.assertFalse(
+            "Returns false if scheme value is null", 
+            Utils.isZkAuthenticationConfiguredTopology(topologyMockMap(null)));
 
-    @Test
-    public void isZkAuthenticationConfiguredTopologyReturnsTrueWhenSchemeSetToStringTest(){
-        Assert.assertTrue(Utils.isZkAuthenticationConfiguredTopology(topologyMockMap("foobar")));
+        Assert.assertTrue(
+            "Returns true if scheme value is string", 
+            Utils.isZkAuthenticationConfiguredTopology(topologyMockMap("foobar")));
     }
 
-    /* isZkAuthenticationConfiguredStormServer */
     @Test
-    public void isZkAuthenticationConfiguredStormReturnsFalseOnNullConfigTest(){
-        Assert.assertFalse(Utils.isZkAuthenticationConfiguredStormServer(null));
-    }
+    public void isZkAuthenticationConfiguredStormServerTest() {
+        Assert.assertFalse(
+            "Returns false if given null config",
+            Utils.isZkAuthenticationConfiguredStormServer(null));
 
-    @Test
-    public void isZkAuthenticationConfiguredStormReturnsFalseOnSchemeKeyMissingTest(){
-        Assert.assertFalse(Utils.isZkAuthenticationConfiguredStormServer(emptyMockMap()));
-    }
+        Assert.assertFalse(
+            "Returns false if scheme key is missing",
+            Utils.isZkAuthenticationConfiguredStormServer(emptyMockMap()));
 
-    @Test
-    public void isZkAuthenticationConfiguredStormReturnsFalseOnSchemeValueNullTest(){
-        Assert.assertFalse(Utils.isZkAuthenticationConfiguredStormServer(serverMockMap(null)));
-    }
+        Assert.assertFalse(
+            "Returns false if scheme value is null",
+            Utils.isZkAuthenticationConfiguredStormServer(serverMockMap(null)));
 
-    @Test
-    public void isZkAuthenticationConfiguredStormReturnsTrueWhenSchemeSetToStringTest(){
-        Assert.assertTrue(Utils.isZkAuthenticationConfiguredStormServer(serverMockMap("foobar")));
+        Assert.assertTrue(
+            "Returns true if scheme value is string",
+            Utils.isZkAuthenticationConfiguredStormServer(serverMockMap("foobar")));
     }
 
     @Test
-    public void isZkAuthenticationConfiguredStormReturnsTrueWhenAuthLoginConfigIsSetTest(){
+    public void isZkAuthenticationConfiguredStormServerWithPropertyTest() {
         String key = "java.security.auth.login.config";
         String oldValue = System.getProperty(key);
         try {
@@ -133,7 +127,7 @@ public class UtilsTest{
         } catch (Exception ignore) {
         } finally {
             // reset property
-            if (oldValue == null){
+            if (oldValue == null) {
                 System.clearProperty(key);
             } else {
                 System.setProperty(key, oldValue);
@@ -141,14 +135,14 @@ public class UtilsTest{
         }
     }
 
-    private CuratorFrameworkFactory.Builder setupBuilder(boolean withExhibitor){
+    private CuratorFrameworkFactory.Builder setupBuilder(boolean withExhibitor) {
         return setupBuilder(withExhibitor, false /*without Auth*/);
     }
 
-    private CuratorFrameworkFactory.Builder setupBuilder(boolean withExhibitor, boolean withAuth){
+    private CuratorFrameworkFactory.Builder setupBuilder(boolean withExhibitor, boolean withAuth) {
         CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder();
         Map<String, Object> conf = new HashMap<String, Object>();
-        if (withExhibitor){
+        if (withExhibitor) {
             conf.put(Config.STORM_EXHIBITOR_SERVERS,"foo");
             conf.put(Config.STORM_EXHIBITOR_PORT, 0);
             conf.put(Config.STORM_EXHIBITOR_URIPATH, "/exhibitor");
@@ -164,7 +158,7 @@ public class UtilsTest{
         conf.put(Config.STORM_ZOOKEEPER_RETRY_TIMES, 0);
         String zkStr = new String("zk_connection_string");
         ZookeeperAuthInfo auth = null;
-        if (withAuth){
+        if (withAuth) {
             auth = new ZookeeperAuthInfo("scheme", "abc".getBytes());
         }
         Utils.testSetupBuilder(builder, zkStr, conf, auth);
@@ -172,21 +166,21 @@ public class UtilsTest{
     }
 
     @Test
-    public void ifExhibitorServersProvidedBuilderUsesTheExhibitorEnsembleProviderTest(){
+    public void givenExhibitorServersBuilderUsesExhibitorProviderTest() {
         CuratorFrameworkFactory.Builder builder = setupBuilder(true /*with exhibitor*/);
         Assert.assertEquals(builder.getEnsembleProvider().getConnectionString(), "");
         Assert.assertEquals(builder.getEnsembleProvider().getClass(), ExhibitorEnsembleProvider.class);
     }
 
     @Test
-    public void ifExhibitorServersAreEmptyBuilderUsesAFixedEnsembleProviderTest(){
+    public void givenNoExhibitorServersBuilderUsesFixedProviderTest() {
         CuratorFrameworkFactory.Builder builder = setupBuilder(false /*without exhibitor*/);
         Assert.assertEquals(builder.getEnsembleProvider().getConnectionString(), "zk_connection_string");
         Assert.assertEquals(builder.getEnsembleProvider().getClass(), FixedEnsembleProvider.class);
     }
 
     @Test
-    public void ifAuthSchemeAndPayloadAreDefinedBuilderUsesAuthTest(){
+    public void givenSchemeAndPayloadBuilderUsesAuthTest() {
         CuratorFrameworkFactory.Builder builder = setupBuilder(false /*without exhibitor*/, true /*with auth*/);
         List<AuthInfo> authInfos = builder.getAuthInfos();
         AuthInfo authInfo = authInfos.get(0);
@@ -195,27 +189,25 @@ public class UtilsTest{
     }
 
     @Test
-    public void parseJvmHeapMemByChildOpts1024KIs1Test(){
-        Assert.assertEquals(Utils.parseJvmHeapMemByChildOpts("Xmx1024K", 0.0).doubleValue(), 1.0, 0); 
-    }
+    public void parseJvmHeapMemByChildOptsTest() {
+        Assert.assertEquals(
+            "1024K results in 1 MB",
+            Utils.parseJvmHeapMemByChildOpts("Xmx1024K", 0.0).doubleValue(), 1.0, 0); 
 
-    @Test
-    public void parseJvmHeapMemByChildOpts100MIs100Test(){
-        Assert.assertEquals(Utils.parseJvmHeapMemByChildOpts("Xmx100M", 0.0).doubleValue(), 100.0, 0); 
-    }
+        Assert.assertEquals(
+            "100M results in 100 MB",
+            Utils.parseJvmHeapMemByChildOpts("Xmx100M", 0.0).doubleValue(), 100.0, 0); 
 
-    @Test
-    public void parseJvmHeapMemByChildOpts1GIs1024Test(){
-        Assert.assertEquals(Utils.parseJvmHeapMemByChildOpts("Xmx1G", 0.0).doubleValue(), 1024.0, 0); 
-    }
+        Assert.assertEquals(
+            "1G results in 1024 MB",
+            Utils.parseJvmHeapMemByChildOpts("Xmx1G", 0.0).doubleValue(), 1024.0, 0); 
 
-    @Test
-    public void parseJvmHeapMemByChildOptsReturnsDefaultIfMatchNotFoundTest(){
-        Assert.assertEquals(Utils.parseJvmHeapMemByChildOpts("Xmx1T", 123.0).doubleValue(), 123.0, 0); 
-    }
+        Assert.assertEquals(
+            "Unmatched value results in default",
+            Utils.parseJvmHeapMemByChildOpts("Xmx1T", 123.0).doubleValue(), 123.0, 0); 
 
-    @Test
-    public void parseJvmHeapMemByChildOptsReturnsDefaultIfInputIsNullTest(){
-        Assert.assertEquals(Utils.parseJvmHeapMemByChildOpts(null, 123.0).doubleValue(), 123.0, 0); 
+        Assert.assertEquals(
+            "Null value results in default",
+            Utils.parseJvmHeapMemByChildOpts(null, 123.0).doubleValue(), 123.0, 0); 
     }
 }


[3/8] storm git commit: STORM-1255: combine two tests to make things clearer

Posted by bo...@apache.org.
STORM-1255: combine two tests to make things clearer


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

Branch: refs/heads/master
Commit: 3fe11ecc652790684010b5fdd36c53844e89ce42
Parents: 20851f8
Author: Alessandro Bellina <ab...@yahoo-inc.com>
Authored: Wed Feb 17 12:31:58 2016 -0600
Committer: Alessandro Bellina <ab...@yahoo-inc.com>
Committed: Wed Feb 17 12:31:58 2016 -0600

----------------------------------------------------------------------
 storm-core/test/jvm/org/apache/storm/utils/TimeTest.java | 8 ++------
 1 file changed, 2 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/3fe11ecc/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java b/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java
index eb5e1d5..13b4914 100644
--- a/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java
+++ b/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java
@@ -33,18 +33,14 @@ public class TimeTest {
         Assert.assertEquals(Time.secsToMillisLong(10.1),  10100);
     }
 
-    @Test
-    public void ifNotSimulatingIsSimulatingReturnsFalse() {
-        Assert.assertFalse(Time.isSimulating());
-    }
-
     @Test(expected=IllegalStateException.class)
     public void ifNotSimulatingAdvanceTimeThrows() {
         Time.advanceTime(1000);
     }
 
     @Test
-    public void ifSimulatingIsSimulatingReturnsTrue() {
+    public void isSimulatingReturnsTrueDuringSimulationTest() {
+        Assert.assertFalse(Time.isSimulating());
         Time.startSimulating();
         Assert.assertTrue(Time.isSimulating());
         Time.stopSimulating();


[5/8] storm git commit: STORM-1255: address pr comments

Posted by bo...@apache.org.
STORM-1255: address pr comments


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

Branch: refs/heads/master
Commit: d912d50a7fcb82883543e004f801490cf41865a2
Parents: a8edd51
Author: Alessandro Bellina <ab...@yahoo-inc.com>
Authored: Thu Feb 18 22:15:42 2016 -0600
Committer: Alessandro Bellina <ab...@yahoo-inc.com>
Committed: Thu Feb 18 22:15:42 2016 -0600

----------------------------------------------------------------------
 .../src/jvm/org/apache/storm/utils/Time.java    |  1 +
 .../jvm/org/apache/storm/utils/TimeTest.java    | 52 +++++++++++---------
 .../jvm/org/apache/storm/utils/UtilsTest.java   | 14 ++++--
 3 files changed, 41 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/d912d50a/storm-core/src/jvm/org/apache/storm/utils/Time.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/utils/Time.java b/storm-core/src/jvm/org/apache/storm/utils/Time.java
index fd01fb8..1b36070 100644
--- a/storm-core/src/jvm/org/apache/storm/utils/Time.java
+++ b/storm-core/src/jvm/org/apache/storm/utils/Time.java
@@ -127,6 +127,7 @@ public class Time {
     
     public static void advanceTime(long ms) {
         if(!simulating.get()) throw new IllegalStateException("Cannot simulate time unless in simulation mode");
+        if(ms < 0) throw new IllegalArgumentException("advanceTime only accepts positive time as an argument");
         simulatedCurrTimeMs.set(simulatedCurrTimeMs.get() + ms);
     }
     

http://git-wip-us.apache.org/repos/asf/storm/blob/d912d50a/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java b/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java
index 354095c..d27b4b8 100644
--- a/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java
+++ b/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java
@@ -34,7 +34,7 @@ public class TimeTest {
     }
 
     @Test(expected=IllegalStateException.class)
-    public void ifNotSimulatingAdvanceTimeThrows() {
+    public void ifNotSimulatingAdvanceTimeThrowsTest() {
         Time.advanceTime(1000);
     }
 
@@ -42,39 +42,47 @@ public class TimeTest {
     public void isSimulatingReturnsTrueDuringSimulationTest() {
         Assert.assertFalse(Time.isSimulating());
         Time.startSimulating();
-        Assert.assertTrue(Time.isSimulating());
-        Time.stopSimulating();
+        try {
+            Assert.assertTrue(Time.isSimulating());
+        } finally {
+            Time.stopSimulating();
+        }
     }
 
     @Test
     public void shouldNotAdvanceTimeTest() {
         Time.startSimulating();
-        long current = Time.currentTimeMillis();
-        Time.advanceTime(0);
-        Assert.assertEquals(Time.deltaMs(current), 0);
-        Time.stopSimulating();
+        try{
+            long current = Time.currentTimeMillis();
+            Time.advanceTime(0);
+            Assert.assertEquals(Time.deltaMs(current), 0);
+        } finally {
+            Time.stopSimulating();
+        }
     }
 
     @Test
     public void shouldAdvanceForwardTest() {
         Time.startSimulating();
-        long current = Time.currentTimeMillis();
-        Time.advanceTime(1000);
-        Assert.assertEquals(Time.deltaMs(current), 1000);
-        Time.advanceTime(500);
-        Assert.assertEquals(Time.deltaMs(current), 1500);
-        Time.stopSimulating();
+        try {
+            long current = Time.currentTimeMillis();
+            Time.advanceTime(1000);
+            Assert.assertEquals(Time.deltaMs(current), 1000);
+            Time.advanceTime(500);
+            Assert.assertEquals(Time.deltaMs(current), 1500);
+        } finally {
+            Time.stopSimulating();
+        }
     }
 
-    @Test
-    public void shouldAdvanceBackwardsTest() {
+    @Test(expected=IllegalArgumentException.class)
+    public void shouldThrowIfAttemptToAdvanceBackwardsTest() {
         Time.startSimulating();
-        long current = Time.currentTimeMillis();
-        Time.advanceTime(1000);
-        Assert.assertEquals(Time.deltaMs(current), 1000);
-        Time.advanceTime(-1500);
-        Assert.assertEquals(Time.deltaMs(current), -500);
-        Time.stopSimulating();
+        try {
+            Time.advanceTime(-1500);
+        } finally {
+            Time.stopSimulating();
+        }
     }
 
     @Test
@@ -87,7 +95,7 @@ public class TimeTest {
     }
 
     @Test
-    public void deltaSecsTruncatesFractionalSeconds() {
+    public void deltaSecsTruncatesFractionalSecondsTest() {
         Time.startSimulating();
         int current = Time.currentTimeSecs();
         Time.advanceTime(1500);

http://git-wip-us.apache.org/repos/asf/storm/blob/d912d50a/storm-core/test/jvm/org/apache/storm/utils/UtilsTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/utils/UtilsTest.java b/storm-core/test/jvm/org/apache/storm/utils/UtilsTest.java
index 8583a16..a74522a 100644
--- a/storm-core/test/jvm/org/apache/storm/utils/UtilsTest.java
+++ b/storm-core/test/jvm/org/apache/storm/utils/UtilsTest.java
@@ -54,11 +54,18 @@ public class UtilsTest {
         Assert.assertEquals(policy.getSleepTimeMs(10, 0), expectedCeiling);
     }
 
-    @Test(expected = RuntimeException.class)
-    public void getConfiguredClientThrowsRuntimeExceptionOnBadArgsTest () throws RuntimeException, TTransportException {
+    public void getConfiguredClientThrowsRuntimeExceptionOnBadArgsTest () throws TTransportException {
         Map config = ConfigUtils.readStormConfig();
         config.put(Config.STORM_NIMBUS_RETRY_TIMES, 0);
-        new NimbusClient(config, "", 65535);
+
+        try {
+            new NimbusClient(config, "", 65535);
+            Assert.fail("Expected exception to be thrown");
+        } catch (RuntimeException e){
+            Assert.assertTrue(
+                "Cause is not TTransportException " + e,  
+                Utils.exceptionCauseIsInstanceOf(TTransportException.class, e));
+        }
     }
 
     private Map mockMap(String key, String value) {
@@ -124,7 +131,6 @@ public class UtilsTest {
         try {
             System.setProperty("java.security.auth.login.config", "anything");
             Assert.assertTrue(Utils.isZkAuthenticationConfiguredStormServer(emptyMockMap()));
-        } catch (Exception ignore) {
         } finally {
             // reset property
             if (oldValue == null) {


[4/8] storm git commit: STORM-1255: fix spacing in test

Posted by bo...@apache.org.
STORM-1255: fix spacing in test


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

Branch: refs/heads/master
Commit: a8edd512c903d44bf26a57cace8f848cc4660738
Parents: 3fe11ec
Author: Alessandro Bellina <ab...@yahoo-inc.com>
Authored: Wed Feb 17 12:36:32 2016 -0600
Committer: Alessandro Bellina <ab...@yahoo-inc.com>
Committed: Wed Feb 17 12:36:32 2016 -0600

----------------------------------------------------------------------
 storm-core/test/jvm/org/apache/storm/utils/TimeTest.java | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/a8edd512/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java b/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java
index 13b4914..354095c 100644
--- a/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java
+++ b/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java
@@ -25,12 +25,12 @@ public class TimeTest {
 
     @Test
     public void secsToMillisLongTest() {
-        Assert.assertEquals(Time.secsToMillisLong(0),     0);
+        Assert.assertEquals(Time.secsToMillisLong(0), 0);
         Assert.assertEquals(Time.secsToMillisLong(0.002), 2);
-        Assert.assertEquals(Time.secsToMillisLong(1),     1000);
-        Assert.assertEquals(Time.secsToMillisLong(1.08),  1080);
-        Assert.assertEquals(Time.secsToMillisLong(10),    10000);
-        Assert.assertEquals(Time.secsToMillisLong(10.1),  10100);
+        Assert.assertEquals(Time.secsToMillisLong(1), 1000);
+        Assert.assertEquals(Time.secsToMillisLong(1.08), 1080);
+        Assert.assertEquals(Time.secsToMillisLong(10), 10000);
+        Assert.assertEquals(Time.secsToMillisLong(10.1), 10100);
     }
 
     @Test(expected=IllegalStateException.class)


[6/8] storm git commit: STORM-1255: missed a couple of finally

Posted by bo...@apache.org.
STORM-1255: missed a couple of finally


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

Branch: refs/heads/master
Commit: 0ca49ab4fdb73ad8b920bb9c8a942fefebd6543c
Parents: d912d50
Author: Alessandro Bellina <ab...@yahoo-inc.com>
Authored: Thu Feb 18 22:17:25 2016 -0600
Committer: Alessandro Bellina <ab...@yahoo-inc.com>
Committed: Thu Feb 18 22:17:25 2016 -0600

----------------------------------------------------------------------
 .../jvm/org/apache/storm/utils/TimeTest.java    | 22 +++++++++++++-------
 1 file changed, 14 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/0ca49ab4/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java b/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java
index d27b4b8..f4b3977 100644
--- a/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java
+++ b/storm-core/test/jvm/org/apache/storm/utils/TimeTest.java
@@ -88,19 +88,25 @@ public class TimeTest {
     @Test
     public void deltaSecsConvertsToSecondsTest() {
         Time.startSimulating();
-        int current = Time.currentTimeSecs();
-        Time.advanceTime(1000);
-        Assert.assertEquals(Time.deltaSecs(current), 1);
-        Time.stopSimulating();
+        try {
+            int current = Time.currentTimeSecs();
+            Time.advanceTime(1000);
+            Assert.assertEquals(Time.deltaSecs(current), 1);
+        } finally {
+            Time.stopSimulating();
+        }
     }
 
     @Test
     public void deltaSecsTruncatesFractionalSecondsTest() {
         Time.startSimulating();
-        int current = Time.currentTimeSecs();
-        Time.advanceTime(1500);
-        Assert.assertEquals(Time.deltaSecs(current), 1, 0);
-        Time.stopSimulating();
+        try {
+            int current = Time.currentTimeSecs();
+            Time.advanceTime(1500);
+            Assert.assertEquals(Time.deltaSecs(current), 1, 0);
+        } finally {
+            Time.stopSimulating();
+        }
     }
 
 }


[7/8] storm git commit: Merge branch 'STORM-1255_port_utils_test_to_java' of https://github.com/abellina/storm into STORM-1255

Posted by bo...@apache.org.
Merge branch 'STORM-1255_port_utils_test_to_java' of https://github.com/abellina/storm into STORM-1255

STORM-1255: port storm_utils.clj to java and split Time tests into its


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

Branch: refs/heads/master
Commit: bc33bc25b2b3da1ed7b0960119167e8421afee49
Parents: 58050a5 0ca49ab
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Tue Feb 23 14:08:10 2016 -0600
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Tue Feb 23 14:08:10 2016 -0600

----------------------------------------------------------------------
 .../src/jvm/org/apache/storm/utils/Time.java    |   1 +
 .../src/jvm/org/apache/storm/utils/Utils.java   |  17 +-
 .../test/clj/org/apache/storm/utils_test.clj    | 111 ----------
 .../jvm/org/apache/storm/utils/TimeTest.java    | 112 ++++++++++
 .../jvm/org/apache/storm/utils/UtilsTest.java   | 219 +++++++++++++++++++
 5 files changed, 342 insertions(+), 118 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/bc33bc25/storm-core/src/jvm/org/apache/storm/utils/Utils.java
----------------------------------------------------------------------


[8/8] storm git commit: Added STORM-1255 to Changelog

Posted by bo...@apache.org.
Added STORM-1255 to Changelog


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

Branch: refs/heads/master
Commit: 53446108bfb286edc527449c5c62820c200cf757
Parents: bc33bc2
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Tue Feb 23 14:09:17 2016 -0600
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Tue Feb 23 14:09:17 2016 -0600

----------------------------------------------------------------------
 CHANGELOG.md    | 1 +
 README.markdown | 3 ++-
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/53446108/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index d422fa1..7d4f3d3 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 ## 2.0.0
+ * STORM-1255: port storm_utils.clj to java and split Time tests into its
  * STORM-1566: Worker exits with error o.a.s.d.worker [ERROR] Error on initialization of server mk-worker
  * STORM-1558: Utils in java breaks component page due to illegal type cast
  * STORM-1553: port event.clj to java

http://git-wip-us.apache.org/repos/asf/storm/blob/53446108/README.markdown
----------------------------------------------------------------------
diff --git a/README.markdown b/README.markdown
index 13e5f2d..3a7e9ad 100644
--- a/README.markdown
+++ b/README.markdown
@@ -253,7 +253,8 @@ under the License.
 * Aaron Dixon ([@atdixon](https://github.com/atdixon))
 * Roshan Naik ([@roshannaik](https://github.com/roshannaik))
 * John Fang ([@hustfxj](https://github.com/hustfxj))
-* Dan Bahir([#dbahir](https://github.com/dbahir))
+* Dan Bahir ([#dbahir](https://github.com/dbahir))
+* Alessandro Bellina ([#abellina](https://github.com/abellina))
 
 ## Acknowledgements