You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@storm.apache.org by revans2 <gi...@git.apache.org> on 2018/01/08 19:23:35 UTC
[GitHub] storm pull request #2507: STORM-2885: Avoid conflicts with nimbusDaemon Loca...
GitHub user revans2 opened a pull request:
https://github.com/apache/storm/pull/2507
STORM-2885: Avoid conflicts with nimbusDaemon LocalCluster Tests
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/revans2/incubator-storm STORM-2885
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/storm/pull/2507.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #2507
----
commit 8f70777bee43d8add02d659d689076c15540976c
Author: Robert (Bobby) Evans <ev...@...>
Date: 2018-01-08T19:21:44Z
STORM-2885: Avoid conflicts with nimbusDaemon LocalCluster Tests
----
---
[GitHub] storm pull request #2507: STORM-2885: Avoid conflicts with nimbusDaemon Loca...
Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:
https://github.com/apache/storm/pull/2507#discussion_r160271224
--- Diff: storm-server/src/test/java/org/apache/storm/nimbus/LocalNimbusTest.java ---
@@ -32,42 +37,35 @@
import org.junit.Assert;
import org.junit.Test;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.UUID;
-
/**
* Tests local cluster with nimbus and a plugin for {@link Config#STORM_TOPOLOGY_SUBMISSION_NOTIFIER_PLUGIN}.
*/
public class LocalNimbusTest {
@Test
public void testSubmitTopologyToLocalNimbus() throws Exception {
-
- HashMap<String,Object> localClusterConf = new HashMap<>();
- localClusterConf.put("nimbus-daemon", true);
- ILocalCluster localCluster = Testing.getLocalCluster(localClusterConf);
-
- Config topoConf = new Config();
- topoConf.putAll(Utils.readDefaultConfig());
- topoConf.setDebug(true);
- topoConf.put("storm.cluster.mode", "local"); // default is aways "distributed" but here local cluster is being used.
- topoConf.put(Config.STORM_TOPOLOGY_SUBMISSION_NOTIFIER_PLUGIN, InmemoryTopologySubmitterHook.class.getName());
-
- List<TopologyDetails> topologyNames =new ArrayList<>();
- for (int i=0; i<4; i++) {
- final String topologyName = "word-count-"+ UUID.randomUUID().toString();
- final StormTopology stormTopology = createTestTopology();
- topologyNames.add(new TopologyDetails(topologyName, stormTopology));
- localCluster.submitTopology(topologyName, topoConf, stormTopology);
+ int port = Utils.getAvailablePort();
--- End diff --
I'd like us to at least file a JIRA to look into changing it, even if we don't do it now. Most of the other uses of Utils.getAvailablePort should have been removed in earlier PRs (e.g. https://github.com/apache/storm/pull/2140). I want to say assigning ports this way was causing test failures on Travis, but it's been a while so I may be misremembering.
---
[GitHub] storm pull request #2507: STORM-2885: Avoid conflicts with nimbusDaemon Loca...
Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:
https://github.com/apache/storm/pull/2507
---
[GitHub] storm pull request #2507: STORM-2885: Avoid conflicts with nimbusDaemon Loca...
Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:
https://github.com/apache/storm/pull/2507#discussion_r160278186
--- Diff: storm-server/src/test/java/org/apache/storm/nimbus/LocalNimbusTest.java ---
@@ -32,42 +37,35 @@
import org.junit.Assert;
import org.junit.Test;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.UUID;
-
/**
* Tests local cluster with nimbus and a plugin for {@link Config#STORM_TOPOLOGY_SUBMISSION_NOTIFIER_PLUGIN}.
*/
public class LocalNimbusTest {
@Test
public void testSubmitTopologyToLocalNimbus() throws Exception {
-
- HashMap<String,Object> localClusterConf = new HashMap<>();
- localClusterConf.put("nimbus-daemon", true);
- ILocalCluster localCluster = Testing.getLocalCluster(localClusterConf);
-
- Config topoConf = new Config();
- topoConf.putAll(Utils.readDefaultConfig());
- topoConf.setDebug(true);
- topoConf.put("storm.cluster.mode", "local"); // default is aways "distributed" but here local cluster is being used.
- topoConf.put(Config.STORM_TOPOLOGY_SUBMISSION_NOTIFIER_PLUGIN, InmemoryTopologySubmitterHook.class.getName());
-
- List<TopologyDetails> topologyNames =new ArrayList<>();
- for (int i=0; i<4; i++) {
- final String topologyName = "word-count-"+ UUID.randomUUID().toString();
- final StormTopology stormTopology = createTestTopology();
- topologyNames.add(new TopologyDetails(topologyName, stormTopology));
- localCluster.submitTopology(topologyName, topoConf, stormTopology);
+ int port = Utils.getAvailablePort();
--- End diff --
Thanks
---
[GitHub] storm pull request #2507: STORM-2885: Avoid conflicts with nimbusDaemon Loca...
Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:
https://github.com/apache/storm/pull/2507#discussion_r160277946
--- Diff: storm-server/src/test/java/org/apache/storm/nimbus/LocalNimbusTest.java ---
@@ -32,42 +37,35 @@
import org.junit.Assert;
import org.junit.Test;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.UUID;
-
/**
* Tests local cluster with nimbus and a plugin for {@link Config#STORM_TOPOLOGY_SUBMISSION_NOTIFIER_PLUGIN}.
*/
public class LocalNimbusTest {
@Test
public void testSubmitTopologyToLocalNimbus() throws Exception {
-
- HashMap<String,Object> localClusterConf = new HashMap<>();
- localClusterConf.put("nimbus-daemon", true);
- ILocalCluster localCluster = Testing.getLocalCluster(localClusterConf);
-
- Config topoConf = new Config();
- topoConf.putAll(Utils.readDefaultConfig());
- topoConf.setDebug(true);
- topoConf.put("storm.cluster.mode", "local"); // default is aways "distributed" but here local cluster is being used.
- topoConf.put(Config.STORM_TOPOLOGY_SUBMISSION_NOTIFIER_PLUGIN, InmemoryTopologySubmitterHook.class.getName());
-
- List<TopologyDetails> topologyNames =new ArrayList<>();
- for (int i=0; i<4; i++) {
- final String topologyName = "word-count-"+ UUID.randomUUID().toString();
- final StormTopology stormTopology = createTestTopology();
- topologyNames.add(new TopologyDetails(topologyName, stormTopology));
- localCluster.submitTopology(topologyName, topoConf, stormTopology);
+ int port = Utils.getAvailablePort();
--- End diff --
I filed STORM-2886 to address the issue.
---
[GitHub] storm pull request #2507: STORM-2885: Avoid conflicts with nimbusDaemon Loca...
Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:
https://github.com/apache/storm/pull/2507#discussion_r160260170
--- Diff: storm-server/src/test/java/org/apache/storm/nimbus/LocalNimbusTest.java ---
@@ -32,42 +37,35 @@
import org.junit.Assert;
import org.junit.Test;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.UUID;
-
/**
* Tests local cluster with nimbus and a plugin for {@link Config#STORM_TOPOLOGY_SUBMISSION_NOTIFIER_PLUGIN}.
*/
public class LocalNimbusTest {
@Test
public void testSubmitTopologyToLocalNimbus() throws Exception {
-
- HashMap<String,Object> localClusterConf = new HashMap<>();
- localClusterConf.put("nimbus-daemon", true);
- ILocalCluster localCluster = Testing.getLocalCluster(localClusterConf);
-
- Config topoConf = new Config();
- topoConf.putAll(Utils.readDefaultConfig());
- topoConf.setDebug(true);
- topoConf.put("storm.cluster.mode", "local"); // default is aways "distributed" but here local cluster is being used.
- topoConf.put(Config.STORM_TOPOLOGY_SUBMISSION_NOTIFIER_PLUGIN, InmemoryTopologySubmitterHook.class.getName());
-
- List<TopologyDetails> topologyNames =new ArrayList<>();
- for (int i=0; i<4; i++) {
- final String topologyName = "word-count-"+ UUID.randomUUID().toString();
- final StormTopology stormTopology = createTestTopology();
- topologyNames.add(new TopologyDetails(topologyName, stormTopology));
- localCluster.submitTopology(topologyName, topoConf, stormTopology);
+ int port = Utils.getAvailablePort();
--- End diff --
Oh we also explicitly check that the port is not 0 to avoid having users shoot themselves in the foot and bring up a nimbus that no one can talk to.
---
[GitHub] storm pull request #2507: STORM-2885: Avoid conflicts with nimbusDaemon Loca...
Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:
https://github.com/apache/storm/pull/2507#discussion_r160248987
--- Diff: storm-server/src/test/java/org/apache/storm/nimbus/LocalNimbusTest.java ---
@@ -32,42 +37,35 @@
import org.junit.Assert;
import org.junit.Test;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.UUID;
-
/**
* Tests local cluster with nimbus and a plugin for {@link Config#STORM_TOPOLOGY_SUBMISSION_NOTIFIER_PLUGIN}.
*/
public class LocalNimbusTest {
@Test
public void testSubmitTopologyToLocalNimbus() throws Exception {
-
- HashMap<String,Object> localClusterConf = new HashMap<>();
- localClusterConf.put("nimbus-daemon", true);
- ILocalCluster localCluster = Testing.getLocalCluster(localClusterConf);
-
- Config topoConf = new Config();
- topoConf.putAll(Utils.readDefaultConfig());
- topoConf.setDebug(true);
- topoConf.put("storm.cluster.mode", "local"); // default is aways "distributed" but here local cluster is being used.
- topoConf.put(Config.STORM_TOPOLOGY_SUBMISSION_NOTIFIER_PLUGIN, InmemoryTopologySubmitterHook.class.getName());
-
- List<TopologyDetails> topologyNames =new ArrayList<>();
- for (int i=0; i<4; i++) {
- final String topologyName = "word-count-"+ UUID.randomUUID().toString();
- final StormTopology stormTopology = createTestTopology();
- topologyNames.add(new TopologyDetails(topologyName, stormTopology));
- localCluster.submitTopology(topologyName, topoConf, stormTopology);
+ int port = Utils.getAvailablePort();
--- End diff --
Since this is the only test that uses a Nimbus daemon it's probably not going to cause issues yet, but this way of getting a port is racy. Is there a way to instead get the Nimbus port from the running LocalCluster, and configure the Nimbus daemon to start on port 0?
---
[GitHub] storm pull request #2507: STORM-2885: Avoid conflicts with nimbusDaemon Loca...
Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:
https://github.com/apache/storm/pull/2507#discussion_r160259977
--- Diff: storm-server/src/test/java/org/apache/storm/nimbus/LocalNimbusTest.java ---
@@ -32,42 +37,35 @@
import org.junit.Assert;
import org.junit.Test;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.UUID;
-
/**
* Tests local cluster with nimbus and a plugin for {@link Config#STORM_TOPOLOGY_SUBMISSION_NOTIFIER_PLUGIN}.
*/
public class LocalNimbusTest {
@Test
public void testSubmitTopologyToLocalNimbus() throws Exception {
-
- HashMap<String,Object> localClusterConf = new HashMap<>();
- localClusterConf.put("nimbus-daemon", true);
- ILocalCluster localCluster = Testing.getLocalCluster(localClusterConf);
-
- Config topoConf = new Config();
- topoConf.putAll(Utils.readDefaultConfig());
- topoConf.setDebug(true);
- topoConf.put("storm.cluster.mode", "local"); // default is aways "distributed" but here local cluster is being used.
- topoConf.put(Config.STORM_TOPOLOGY_SUBMISSION_NOTIFIER_PLUGIN, InmemoryTopologySubmitterHook.class.getName());
-
- List<TopologyDetails> topologyNames =new ArrayList<>();
- for (int i=0; i<4; i++) {
- final String topologyName = "word-count-"+ UUID.randomUUID().toString();
- final StormTopology stormTopology = createTestTopology();
- topologyNames.add(new TopologyDetails(topologyName, stormTopology));
- localCluster.submitTopology(topologyName, topoConf, stormTopology);
+ int port = Utils.getAvailablePort();
--- End diff --
Yes it is racy.
We have done it this way for other ports without too many issues. The problem is that it would require a feedback path for the port from Nimbus, and possibly a few updates in other places that try to read the port out of the config. If you really want to see this changed I can file a follow on JIRA and look into it, as it would not likely be that much work.
---