[
https://issues.apache.org/jira/browse/STORM-166?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14377294#comment-14377294
]
ASF GitHub Bot commented on STORM-166:
--------------------------------------
Github user Parth-Brahmbhatt commented on the pull request:
https://github.com/apache/storm/pull/354#issuecomment-85338889
Disclaimer I did not thoroughly look at the code but I am commenting based
on your design description of Jstorm.
@longdafeng Did you have a chance to take a look at the current design? We
are using curator for leader election which seems to be a very well tested
library and is not really far from what you have proposed for leader election.
As for the length of the code, I don't completely agree with that being a
good metric for most things. Due to the usage of an existing library the actual
code for leader election in current PR is much smaller, 53 lines.
https://github.com/Parth-Brahmbhatt/incubator-storm/blob/STORM-166/storm-core/src/clj/backtype/storm/zookeeper.clj#L250.
On top of that as part of this PR several of us had concerns around all
clients connecting to zk to identify leader nimbus , as each new zk connection
is a write to zk. We have partially fixed the issue by introducing thrift APIs
for nimbus discovery which should be more efficient then the original approach
and I plan to add caching at nimbus layer which should further improve the
performance.
As @ptgoetz mentioned in the jira, we do not want user's topologies
getting lost once nimbus accepts it and we also do not want to force all users
to have a dependency on a fully replicated storage layer like HDFS. In current
design by adding a code replication interface we are guaranteeing that once a
topology is in active state it will be fully replicated, which seems to be
another missing feature in your proposal. Its still a choice between
availability and initial topology submission time which the users can chose
based on their topology.replication.count config setting.
We also added few more features like UI improvements, nimbus summary being
stored in zk, thrift API modification so users can figure out replication
factor of their topologies, compatibility with rolling upgrade feature. All of
which in my opinion are good admin tools and this feature will be incomplete
without it.
I appreciate any feedback you can provide based on your experience of
running Nimbus HA in production for a year. Please take some time to review the
current design and let us know if you have any concerns.
> Highly available Nimbus
> -----------------------
>
> Key: STORM-166
> URL: https://issues.apache.org/jira/browse/STORM-166
> Project: Apache Storm
> Issue Type: New Feature
> Reporter: James Xu
> Assignee: Parth Brahmbhatt
> Priority: Minor
>
> https://github.com/nathanmarz/storm/issues/360
> The goal of this feature is to be able to run multiple Nimbus servers so that
> if one goes down another one will transparently take over. Here's what needs
> to happen to implement this:
> 1. Everything currently stored on local disk on Nimbus needs to be stored in
> a distributed and reliable fashion. A DFS is perfect for this. However, as we
> do not want to make a DFS a mandatory requirement to run Storm, the storage
> of these artifacts should be pluggable (default to local filesystem, but the
> interface should support DFS). You would only be able to run multiple NImbus
> if you use the right storage, and the storage interface chosen should have a
> flag indicating whether it's suitable for HA mode or not. If you choose local
> storage and try to run multiple Nimbus, one of the Nimbus's should fail to
> launch.
> 2. Nimbus's should register themselves in Zookeeper. They should use a leader
> election protocol to decide which one is currently responsible for launching
> and monitoring topologies.
> 3. StormSubmitter should find the Nimbus to connect to via Zookeeper. In case
> the leader changes during submission, it should use a retry protocol to try
> reconnecting to the new leader and attempting submission again.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)