storm-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Stig Rohde Døssing <s...@apache.org>
Subject Re: How to set Zookeeper's data directory & port when running in LocalCluster mode?
Date Sun, 27 Aug 2017 22:16:00 GMT
Sorry, I was looking at the master (2.0.0) files, not 1.x.

1. In 2.0.0 the LocalCluster(Builder) constructor is not deprecated but
suppressing deprecation warnings. You'd be creating the LocalCluster by
calling build() on the Builder (https://github.com/apache/
storm/blob/master/storm-server/src/main/java/org/apache/storm/LocalCluster.
java#L326), which uses the private LocalCluster(Builder) constructor
internally. I'll take a look at the 1.x code soon to check how to do
something similar there.

2. The intent was to add a new method to the Builder to set the Zookeeper
data dir. The new configuration parameter would then be passed to
https://github.com/apache/storm/blob/master/storm-server/src/main/java/org/
apache/storm/LocalCluster.java#L415. Again, I'll need to check how to do
this in 1.x.

3. You are right, the published Javadoc doesn't include LocalCluster. We
should probably fix that.

2017-08-27 23:01 GMT+02:00 Alexandre Vermeerbergen <avermeerbergen@gmail.com
>:

> Hello Stig,
>
> Thank you very much for your fast answer.
> Let me answer your feedbacks:
>
> 1. Setting Zookeeper's configuration using LocalCluster.Builder:
>     Unfortunately, event if LocalCluster.Builder inner class is public, I
> don't see a way to use it in order to create a LocalCluster taking
> advantage of a well crafted LocalCluster.Builder instance. Indeed, method
> LocalCluster(Builder builder)
> is private (and @deprecated).
>    Would you please give an example of how to set Zookeeper's
> configuration on LocalCluster (again I'm just testing with
> ExclamationTopology, nothing complex) ?
>
> 2. Yes I'd be glad to propose some PR, but first I need to understand your
> proposal about setting the Zookeeper configuration through the daemon conf
> map (point 1. above)
>
> 3. I wrote that LocalCluster(String host,int port) constructor is not
> documented because I cant' find this class in any Javadoc of Storm. As
> pointed from http://storm.apache.org/releases/current/index.html, the
> official Storm Javadoc at http://storm.apache.org/
> releases/current/javadocs/index.html doesn't include LocalCluster
> documentation, but only ILocalCluster one, which isn't of great interest
> for running a LocalCluster with modest specific configuration settings.
>
> Hope it clarifies,
> Best regards,
>
> Alexandre Vermeerbergen
>
>
>
> 2017-08-27 21:31 GMT+02:00 Stig Rohde Døssing <srdo@apache.org>:
>
>> You can set the Zookeeper configuration through the daemon conf map when
>> you use a LocalCluster.Builder, which should let you set many of the
>> options you might normally put in storm.yaml
>> https://github.com/apache/storm/blob/master/storm-server/
>> src/main/java/org/apache/storm/LocalCluster.java#L173.
>>
>> I don't believe there is a way to set the data dir for Zookeeper right
>> now. As you noted, it is set to use the system's temporary directory
>> https://github.com/apache/storm/blob/master/storm-server/
>> src/main/java/org/apache/storm/testing/InProcessZookeeper.java#L39.
>> Consider raising an issue if you think we should add a function to set this
>> (could e.g. be a new function on the LocalCluster.Builder). A PR would
>> obviously be appreciated if you're up for it :)
>>
>> I'm not sure what you mean by the LocalCluster(host, port) constructor
>> being undocumented. What kind of documentation do you feel is missing?
>>
>> 2017-08-27 19:10 GMT+02:00 Alexandre Vermeerbergen <
>> avermeerbergen@gmail.com>:
>>
>>> Hello,
>>>
>>> I would like to start topologies in LocalCluster mode for testing
>>> purposes using my own settings for the Zookeeper server started by the
>>> LocalCluster mode.
>>>
>>> For example, I would like Zookeeper to write in
>>> e:\tmp\exclam\stormlocaldir and use port 123456.
>>>
>>> Storm documentation isn't quite verbose about how to set LocalCluster
>>> mode options, unless I missed the appropriate page (in which please, please
>>> let me know at which URL I can find how to pass options to LocalCluster
>>> mode).
>>>
>>> I tried to modify the main() method of  ExclamationTopology classic
>>> Storm sampe to fix Zookeeer properties, like this:
>>>
>>>     public static void main(String[] args) throws Exception {
>>>         TopologyBuilder builder = new TopologyBuilder();
>>>
>>>         builder.setSpout("word", new TestWordSpout(), 10);
>>>         builder.setBolt("exclaim1", new ExclamationBolt(),
>>> 3).shuffleGrouping("word");
>>>         builder.setBolt("exclaim2", new ExclamationBolt(),
>>> 2).shuffleGrouping("exclaim1");
>>>
>>>         Config conf = new Config();
>>>         conf.setDebug(true);
>>>
>>>         conf.put(Config.STORM_LOCAL_DIR, "E:\\tmp\\exclam\\stormlocaldi
>>> r\\");
>>>         conf.put(Config.STORM_ZOOKEEPER_ROOT,
>>> "E:\\tmp\\exclam\\stormlocaldir\\");
>>>         conf.put(Config.STORM_ZOOKEEPER_PORT,12346);
>>>         LocalCluster cluster = new LocalCluster();
>>>         cluster.submitTopology("test", conf, builder.createTopology());
>>>         Utils.sleep(120000);
>>>         cluster.killTopology("test");
>>>         cluster.shutdown();
>>>     }
>>> }
>>>
>>> But to my despair, all data is written into subdirectories
>>> C:\Users\ave\AppData\Local\Temp\<some UUID> instead of to
>>> E:\tmp\exclam\stormlocaldir\ subdirectories.
>>>
>>> Also, during the program's execution, Zookeeper listens on port 2000
>>> instead of listening on port 123456 (I checked this: it's not just a wrong
>>> trace).
>>>
>>> I was able to partially by-pass my problem with data directory by
>>> setting Java's system property which gives the temporary directory path,
>>> but this is a very ugly by-pass:
>>>
>>>     public static void main(String[] args) throws Exception {
>>>         TopologyBuilder builder = new TopologyBuilder();
>>>
>>>         builder.setSpout("word", new TestWordSpout(), 10);
>>>         builder.setBolt("exclaim1", new ExclamationBolt(),
>>> 3).shuffleGrouping("word");
>>>         builder.setBolt("exclaim2", new ExclamationBolt(),
>>> 2).shuffleGrouping("exclaim1");
>>>
>>>         Config conf = new Config();
>>>         conf.setDebug(true);
>>>
>>>         System.setProperty("java.io.tmpdir",
>>> "E:\\tmp\\exclam\\stormlocaldir\\");
>>>         conf.put(Config.STORM_ZOOKEEPER_PORT,12346);
>>>         LocalCluster cluster = new LocalCluster();
>>>         cluster.submitTopology("test", conf, builder.createTopology());
>>>         Utils.sleep(120000);
>>>         cluster.killTopology("test");
>>>         cluster.shutdown();
>>>     }
>>>
>>> But I found no by-pass to set Zookeeper's port.
>>>
>>> I am aware that another by-pass could be to use my own Zookeeper server
>>> and then use the un-documented LocalCluster(host,port) constructor to start
>>> a LocalCluster connect to this Zookeeper server, but frankly... isn't there
>>> a simpler way to setup properties for the Zookeeper server started by
>>> default constructor of LocalCluster ?
>>>
>>> Best regards,
>>> Alexandre Vermeerbergen
>>>
>>>
>>>
>>
>

Mime
View raw message