You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by kn...@apache.org on 2015/12/17 21:47:21 UTC

[1/6] storm git commit: merging distcache a.k.a blobstore design and API usage docs

Repository: storm
Updated Branches:
  refs/heads/master 6cf33a86d -> 0e1c969c2


merging distcache a.k.a blobstore design and API usage docs

formatting changes

sentence changes and spelling errors

formatting changes, sentence completion changes

fixing few more tense errors


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

Branch: refs/heads/master
Commit: 63b2e3d3d4dfd1f0909957b9bf04af5ea7df8bd2
Parents: 79a2a2a
Author: Sanket <sc...@untilservice-lm>
Authored: Mon Dec 7 09:43:19 2015 -0600
Committer: Sanket <sc...@untilservice-lm>
Committed: Mon Dec 14 16:35:53 2015 -0600

----------------------------------------------------------------------
 docs/documentation/distcache-blobstore.md       | 732 +++++++++++++++++++
 docs/documentation/images/hdfs_blobstore.png    | Bin 0 -> 82180 bytes
 docs/documentation/images/local_blobstore.png   | Bin 0 -> 81212 bytes
 .../images/nimbus_ha_blobstore.png              | Bin 0 -> 113991 bytes
 4 files changed, 732 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/63b2e3d3/docs/documentation/distcache-blobstore.md
----------------------------------------------------------------------
diff --git a/docs/documentation/distcache-blobstore.md b/docs/documentation/distcache-blobstore.md
new file mode 100644
index 0000000..2de14bb
--- /dev/null
+++ b/docs/documentation/distcache-blobstore.md
@@ -0,0 +1,732 @@
+# Storm Distributed Cache API
+
+The distributed cache feature in storm is used to efficiently distribute files
+(or blobs, which is the equivalent terminology for a file in the distributed
+cache and is used interchangeably in this document) that are large and can
+change during the lifetime of a topology, such as geo-location data,
+dictionaries, etc. Typical use cases include phrase recognition, entity
+extraction, document classification, URL re-writing, location/address detection
+and so forth. Such files may be several KB to several GB in size. For small
+datasets that don't need dynamic updates, including them in the topology jar
+could be fine. But for large files, the startup times could become very large.
+In these cases, the distributed cache feature can provide fast topology startup,
+especially if the files were previously downloaded for the same submitter and
+are still in the cache. This is useful with frequent deployments, sometimes few
+times a day with updated jars, because the large cached files will remain available
+without changes. The large cached blobs that do not change frequently will
+remain available in the distributed cache.
+
+At the starting time of a topology, the user specifies the set of files the
+topology needs. Once a topology is running, the user at any time can request for
+any file in the distributed cache to be updated with a newer version. The
+updating of blobs happens in an eventual consistency model. If the topology
+needs to know what version of a file it has access to, it is the responsibility
+of the user to find this information out. The files are stored in a cache with
+Least-Recently Used (LRU) eviction policy, where the supervisor decides which
+cached files are no longer needed and can delete them to free disk space. The
+blobs can be compressed, and the user can request the blobs to be uncompressed
+before it accesses them.
+
+## Motivation for Distributed Cache
+* Allows sharing blobs among topologies.
+* Allows updating the blobs from the command line.
+
+## Distributed Cache Implementations
+The current BlobStore interface has the following two implementations
+* LocalFsBlobStore
+* HdfsBlobStore
+
+Appendix A contains the interface for blob store implementation.
+
+## LocalFsBlobStore
+![LocalFsBlobStore](images/local_blobstore.png)
+
+Local file system implementation of Blobstore can be depicted in the above timeline diagram.
+
+There are several stages from blob creation to blob download and corresponding execution of a topology. 
+The main stages can be depicted as follows
+
+### Blob Creation Command
+Blobs in the blobstore can be created through command line using the following command.
+storm blobstore create --file README.txt --acl o::rwa --repl-fctr 4 key1
+The above command creates a blob with a key name “key1” corresponding to the file README.txt. 
+The access given to all users being read, write and admin with a replication factor of 4.
+
+### Topology Submission and Blob Mapping
+Users can submit their topology with the following command. The command includes the 
+topology map configuration. The configuration holds two keys “key1” and “key2” with the 
+key “key1” having a local file name mapping named “blob_file” and it is not compressed.
+
+```
+storm jar /home/y/lib/storm-starter/current/storm-starter-jar-with-dependencies.jar 
+storm.starter.clj.word_count test_topo -c topology.blobstore.map='{"key1":{"localname":"blob_file", "uncompress":"false"},"key2":{}}'
+```
+
+### Blob Creation Process
+The creation of the blob takes place through the interface “ClientBlobStore”. Appendix B contains the “ClientBlobStore” interface. 
+The concrete implementation of this interface is the  “NimbusBlobStore”. In the case of local file system the client makes a 
+call to the nimbus to create the blobs within the local file system. The nimbus uses the local file system implementation to create these blobs. 
+When a user submits a topology, the jar, configuration and code files are uploaded as blobs with the help of blob store. 
+Also, all the other blobs specified by the topology are mapped to it with the help of topology.blobstore.map configuration.
+
+### Blob Download by the Supervisor
+Finally, the blobs corresponding to a topology are downloaded by the supervisor once it receives the assignments from the nimbus through 
+the same “NimbusBlobStore” thrift client that uploaded the blobs. The supervisor downloads the code, jar and conf blobs by calling the 
+“NimbusBlobStore” client directly while the blobs specified in the topology.blobstore.map are downloaded and mapped locally with the help 
+of the Localizer. The Localizer talks to the “NimbusBlobStore” thrift client to download the blobs and adds the blob compression and local 
+blob name mapping logic to suit the implementation of a topology. Once all the blobs have been downloaded the workers are launched to run 
+the topologies.
+
+## HdfsBlobStore
+![HdfsBlobStore](images/hdfs_blobstore.png)
+
+The HdfsBlobStore functionality has a similar implementation and blob creation and download procedure barring how the replication 
+is handled in the two blob store implementations. The replication in HDFS blob store is obvious as HDFS is equipped to handle replication 
+and it requires no state to be stored inside the zookeeper. On the other hand, the local file system blobstore requires the state to be 
+stored on the zookeeper in order for it to work with nimbus HA. Nimbus HA allows the local filesystem to implement the replication feature 
+seamlessly by storing the state in the zookeeper about the running topologies and syncing the blobs on various nimbodes. On the supervisor’s 
+end, the supervisor and localizer talks to HdfsBlobStore through “HdfsClientBlobStore” implementation.
+
+## Additional Features and Documentation
+```
+storm jar /home/y/lib/storm-starter/current/storm-starter-jar-with-dependencies.jar storm.starter.clj.word_count test_topo 
+-c topology.blobstore.map='{"key1":{"localname":"blob_file", "uncompress":"false"},"key2":{}}'
+```
+ 
+### Compression
+The blob store allows the user to specify the “uncompress” configuration to true or false. This configuration can be specified 
+in the topology.blobstore.map mentioned in the above command. This allows the user to upload a compressed file like a tarball/zip. 
+In local file system blob store, the compressed blobs are stored on the nimbus node. The localizer code takes the responsibility to 
+uncompress the blob and store it on the supervisor node. Symbolic links to the blobs on the supervisor node are created within the worker 
+before the execution starts.
+
+### Local File Name Mapping
+Apart from compression the blobstore helps to give the blob a name that can be used by the workers. The localizer takes 
+the responsibility of mapping the blob to a local name on the supervisor node.
+
+## Additional Blob Store Implementation Details
+Blob store uses a hashing function to create the blobs based on the key. The blobs are generally stored inside the directory specified by 
+the blobstore.dir configuration. By default, it is stored under “storm.local.dir/nimbus/blobs” for local file system and a similar path on 
+hdfs file system.
+
+Once a file is submitted, the blob store reads the configs and creates a metadata for the blob with all the access control details. The metadata 
+is generally used for authorization while accessing the blobs. The blob key and version contribute to the hash code and there by the directory 
+under “storm.local.dir/nimbus/blobs/data” where the data is placed. The blobs are generally placed in a positive number directory like 193,822 etc.
+
+Once the topology is launched and the relevant blobs have been created the supervisor downloads blobs related to the storm.conf, storm.ser 
+and storm.code first and all the blobs uploaded by the command line separately using the localizer to uncompress and map them to a local name 
+specified in the topology.blobstore.map configuration. The supervisor periodically updates blobs by checking for the change of version. 
+This allows updating the blobs on the fly and thereby making it a very useful feature.
+
+For a local file system, the distributed cache on the supervisor node is set to 10240 MB as a soft limit and the clean up code attempts 
+to clean anything over the soft limit every 600 seconds based on LRU policy.
+
+The HDFS blob store implementation handles load better by removing the burden on the nimbus to store the blobs, which avoids it becoming a bottleneck. Moreover, it provides seamless replication of blobs. On the other hand, the local file system blob store is not very efficient in 
+replicating the blobs and is limited by the number of nimbuses. Moreover, the supervisor talks to the HDFS blob store directly without the 
+involvement of the nimbus and thereby reduces the load and dependency on nimbus.
+
+## Highly Available Nimbus
+### Problem Statement:
+Currently the storm master aka nimbus, is a process that runs on a single machine under supervision. In most cases the 
+nimbus failure is transient and it is restarted by the supervisor. However sometimes when disks fail and networks 
+partitions occur, nimbus goes down. Under these circumstances the topologies run normally but no new topologies can be 
+submitted, no existing topologies can be killed/deactivated/activated and if a supervisor node fails then the 
+reassignments are not performed resulting in performance degradation or topology failures. With this project we intend 
+to resolve this problem by running nimbus in a primary backup mode to guarantee that even if a nimbus server fails one 
+of the backups will take over. 
+
+### Requirements for Highly Available Nimbus:
+* Increase overall availability of nimbus.
+* Allow nimbus hosts to leave and join the cluster at will any time. A newly joined host should auto catch up and join 
+the list of potential leaders automatically. 
+* No topology resubmissions required in case of nimbus fail overs.
+* No active topology should ever be lost. 
+
+#### Leader Election:
+The nimbus server will use the following interface:
+
+```java
+public interface ILeaderElector {
+    /**
+     * queue up for leadership lock. The call returns immediately and the caller                     
+     * must check isLeader() to perform any leadership action.
+     */
+    void addToLeaderLockQueue();
+
+    /**
+     * Removes the caller from the leader lock queue. If the caller is leader
+     * also releases the lock.
+     */
+    void removeFromLeaderLockQueue();
+
+    /**
+     *
+     * @return true if the caller currently has the leader lock.
+     */
+    boolean isLeader();
+
+    /**
+     *
+     * @return the current leader's address , throws exception if noone has has    lock.
+     */
+    InetSocketAddress getLeaderAddress();
+
+    /**
+     * 
+     * @return list of current nimbus addresses, includes leader.
+     */
+    List<InetSocketAddress> getAllNimbusAddresses();
+}
+```
+Once a nimbus comes up it calls addToLeaderLockQueue() function. The leader election code selects a leader from the queue.
+If the topology code, jar or config blobs are missing, it would download the blobs from any other nimbus which is up and running.
+
+The first implementation will be Zookeeper based. If the zookeeper connection is lost/reset resulting in loss of lock
+or the spot in queue the implementation will take care of updating the state such that isLeader() will reflect the 
+current status.The leader like actions must finish in less than minimumOf(connectionTimeout, SessionTimeout) to ensure
+the lock was held by nimbus for the entire duration of the action (Not sure if we want to just state this expectation 
+and ensure that zk configurations are set high enough which will result in higher failover time or we actually want to 
+create some sort of rollback mechanism for all actions, the second option needs a lot of code). If a nimbus that is not 
+leader receives a request that only a leader can perform it will throw a RunTimeException.
+
+### Nimbus state store:
+
+To achieve fail over from primary to backup servers nimbus state/data needs to be replicated across all nimbus hosts or 
+needs to be stored in a distributed storage. Replicating the data correctly involves state management, consistency checks
+and it is hard to test for correctness. However many storm users do not want to take extra dependency on another replicated
+storage system like HDFS and still need high availability. The blob store implementation along with the state storage helps
+to overcome the failover scenarios in case a leader nimbus goes down.
+
+To support replication we will allow the user to define a code replication factor which would reflect number of nimbus 
+hosts to which the code must be replicated before starting the topology. With replication comes the issue of consistency. 
+The topology is launched once the code, jar and conf blob files are replicated based on the "topology.min.replication" config.
+Maintaining state for failover scenarios is important for local file system. The current implementation makes sure one of the
+available nimbus is elected as a leader in the case of a failure. If the topology specific blobs are missing, the leader nimbus
+tries to download them as and when they are needed. With this current architecture, we do not have to download all the blobs 
+required for a topology for a nimbus to accept leadership. This helps us in case the blobs are very large and avoid causing any 
+inadvertant delays in electing a leader.
+
+The state for every blob is relevant for the local blob store implementation. For HDFS blob store the replication
+is taken care by the HDFS. For handling the fail over scenarios for a local blob store we need to store the state of the leader and
+non-leader nimbodes within the zookeeper.
+
+The state is stored under /storm/blobstore/key/nimbusHostPort:SequenceNumber for the blob store to work to make nimbus highly available. 
+This state is used in the local file system blobstore to support replication. The HDFS blobstore does not have to store the state inside the 
+zookeeper.
+
+* NimbusHostPort: This piece of information generally contains the parsed string holding the hostname and port of the nimbus. 
+  It uses the same class “NimbusHostPortInfo” used earlier by the code-distributor interface to store the state and parse the data.
+
+* SequenceNumber: This is the blob sequence number information. The SequenceNumber information is implemented by a KeySequenceNumber class. 
+The sequence numbers are generated for every key. For every update, the sequence numbers are assigned based ona global sequence number 
+stored under /storm/blobstoremaxsequencenumber/key. For more details about how the numbers are generated you can look at the java docs for KeySequenceNumber.
+
+![Nimbus High Availability - BlobStore](images/nimbus_ha_blobstore.png)
+
+The sequence diagram proposes how the blob store works and the state storage inside the zookeeper makes the nimbus highly available.
+Currently, the thread to sync the blobs on a non-leader is within the nimbus. In the future, it will be nice to move the thread around
+to the blob store to make the blobstore coordinate the state change and blob download as per the sequence diagram.
+
+## Thrift and Rest API 
+In order to avoid workers/supervisors/ui talking to zookeeper for getting master nimbus address we are going to modify the 
+`getClusterInfo` API so it can also return nimbus information. getClusterInfo currently returns `ClusterSummary` instance
+which has a list of `supervisorSummary` and a list of `topologySummary` instances. We will add a list of `NimbusSummary` 
+to the `ClusterSummary`. See the structures below:
+
+```thrift
+struct ClusterSummary {
+  1: required list<SupervisorSummary> supervisors;
+  3: required list<TopologySummary> topologies;
+  4: required list<NimbusSummary> nimbuses;
+}
+
+struct NimbusSummary {
+  1: required string host;
+  2: required i32 port;
+  3: required i32 uptime_secs;
+  4: required bool isLeader;
+  5: required string version;
+}
+```
+
+This will be used by StormSubmitter, Nimbus clients, supervisors and ui to discover the current leaders and participating 
+nimbus hosts. Any nimbus host will be able to respond to these requests. The nimbus hosts can read this information once 
+from zookeeper and cache it and keep updating the cache when the watchers are fired to indicate any changes,which should 
+be rare in general case.
+
+Note: All nimbus hosts have watchers on zookeeper to be notified immediately as soon as a new blobs is available for download, the callback may or may not download
+the code. Therefore, a background thread is triggered to download the respective blobs to run the topologies. The replication is achieved when the blobs are downloaded
+onto non-leader nimbodes. So you should expect your topology submission time to be somewhere between 0 to (2 * nimbus.code.sync.freq.secs) for any 
+nimbus.min.replication.count > 1.
+
+## Configuration
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+blobstore.dir: The directory where all blobs are stored. For local file system it represents the directory on the nimbus 
+node and for HDFS file system it represents the hdfs file system path.
+
+supervisor.blobstore.class: This configuration is meant to set the client for  the supervisor  in order to talk to the blob store. 
+For a local file system blob store it is set to “backtype.storm.blobstore.NimbusBlobStore” and for the HDFS blob store it is set 
+to “backtype.storm.blobstore.HdfsClientBlobStore”.
+
+supervisor.blobstore.download.thread.count: This configuration spawns multiple threads for from the supervisor in order download 
+blobs concurrently. The default is set to 5
+
+supervisor.blobstore.download.max_retries: This configuration is set to allow the supervisor to retry for the blob download. 
+By default it is set to 3.
+
+supervisor.localizer.cache.target.size.mb: The jvm opts provided to workers launched by this supervisor. All "%ID%" substrings 
+are replaced with an identifier for this worker. Also, "%WORKER-ID%", "%STORM-ID%" and "%WORKER-PORT%" are replaced with 
+appropriate runtime values for this worker. The distributed cache target size in MB. This is a soft limit to the size 
+of the distributed cache contents. It is set to 10240 MB.
+
+supervisor.localizer.cleanup.interval.ms: The distributed cache cleanup interval. Controls how often it scans to attempt to 
+cleanup anything over the cache target size. By default it is set to 600000 milliseconds.
+
+nimbus.blobstore.class:  Sets the blobstore implementation nimbus uses. It is set to "backtype.storm.blobstore.LocalFsBlobStore"
+
+nimbus.blobstore.expiration.secs: During operations with the blob store, via master, how long a connection is idle before nimbus 
+considers it dead and drops the session and any associated connections. The default is set to 600.
+
+storm.blobstore.inputstream.buffer.size.bytes: The buffer size it uses for blob store upload. It is set to 65536 bytes.
+
+client.blobstore.class: The blob store implementation the storm client uses. The current implementation uses the default 
+config "backtype.storm.blobstore.NimbusBlobStore".
+
+blobstore.replication.factor: It sets the replication for each blob within the blob store. The “topology.min.replication.count” 
+ensures the minimum replication the topology specific blobs are set before launching the topology. You might want to set the 
+“topology.min.replication.count <= blobstore.replication”. The default is set to 3.
+
+topology.min.replication.count : Minimum number of nimbus hosts where the code must be replicated before leader nimbus
+can mark the topology as active and create assignments. Default is 1.
+
+topology.max.replication.wait.time.sec: Maximum wait time for the nimbus host replication to achieve the nimbus.min.replication.count.
+Once this time is elapsed nimbus will go ahead and perform topology activation tasks even if required nimbus.min.replication.count is not achieved. 
+The default is 60 seconds, a value of -1 indicates to wait for ever.
+* nimbus.code.sync.freq.secs: Frequency at which the background thread on nimbus which syncs code for locally missing blobs. Default is 2 minutes.
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+
+## Using the Distributed Cache API, Command Line Interface (CLI)
+
+### Creating blobs 
+
+To use the distributed cache feature, the user first has to "introduce" files
+that need to be cached and bind them to key strings. To achieve this, the user
+uses the "blobstore create" command of the storm executable, as follows:
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+storm blobstore create [-f|--file FILE] [-a|--acl ACL1,ACL2,...] [--repl-fctr NUMBER] [keyname]
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+The contents come from a FILE, if provided by -f or --file option, otherwise
+from STDIN.  
+The ACLs, which can also be a comma separated list of many ACLs, is of the
+following format:
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+> [u|o]:[username]:[r-|w-|a-|_]
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+where:  
+
+* u = user  
+* o = other  
+* username = user for this particular ACL  
+* r = read access  
+* w = write access  
+* a = admin access  
+* _ = ignored  
+
+The replication factor can be set to a value greater than 1 using --repl-fctr.
+
+Note: The replication right now is configurable for a hdfs blobstore but for a
+local blobstore the replication always stays at 1. For a hdfs blobstore
+the default replication is set to 3.
+
+###### Example:  
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+storm blobstore create --file README.txt --acl o::rwa --repl-fctr 4 key1
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+In the above example, the *README.txt* file is added to the distributed cache.
+It can be accessed using the key string "*key1*" for any topology that needs
+it. The file is set to have read/write/admin access for others, a.k.a world
+everything and the replication is set to 4.
+
+###### Example:  
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+storm blobstore create mytopo:data.tgz -f data.tgz -a u:alice:rwa,u:bob:rw,o::r  
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+The above example createss a mytopo:data.tgz key using the data stored in
+data.tgz.  User alice would have full access, bob would have read/write access
+and everyone else would have read access.
+
+### Making dist. cache files accessible to topologies
+
+Once a blob is created, we can use it for topologies. This is generally achieved
+by including the key string among the configurations of a topology, with the
+following format. A shortcut is to add the configuration item on the command
+line when starting a topology by using the **-c** command:
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+-c topology.blobstore.map='{"[KEY]":{"localname":"[VALUE]", "uncompress":"[true|false]"}}'
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+Note: Please take care of the quotes.
+
+The cache file would then be accessible to the topology as a local file with the
+name [VALUE].  
+The localname parameter is optional, if omitted the local cached file will have
+the same name as [KEY].  
+The uncompress parameter is optional, if omitted the local cached file will not
+be uncompressed.  Note that the key string needs to have the appropriate
+file-name-like format and extension, so it can be uncompressed correctly.
+
+###### Example:  
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+storm jar /home/y/lib/storm-starter/current/storm-starter-jar-with-dependencies.jar storm.starter.clj.word_count test_topo -c topology.blobstore.map='{"key1":{"localname":"blob_file", "uncompress":"false"},"key2":{}}'
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+Note: Please take care of the quotes.
+
+In the above example, we start the *word_count* topology (stored in the
+*storm-starter-jar-with-dependencies.jar* file), and ask it to have access
+to the cached file stored with key string = *key1*. This file would then be
+accessible to the topology as a local file called *blob_file*, and the
+supervisor will not try to uncompress the file. Note that in our example, the
+file's content originally came from *README.txt*. We also ask for the file
+stored with the key string = *key2* to be accessible to the topology. Since
+both the optional parameters are omitted, this file will get the local name =
+*key2*, and will not be uncompressed.
+
+### Updating a cached file
+
+It is possible for the cached files to be updated while topologies are running.
+The update happens in an eventual consistency model, where the supervisors poll
+Nimbus every 30 seconds, and update their local copies. In the current version,
+it is the user's responsibility to check whether a new file is available.
+
+To update a cached file, use the following command. Contents come from a FILE or
+STDIN. Write access is required to be able to update a cached file.
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+storm blobstore update [-f|--file NEW_FILE] [KEYSTRING]
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+###### Example:  
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+storm blobstore update -f updates.txt key1
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+In the above example, the topologies will be presented with the contents of the
+file *updates.txt* instead of *README.txt* (from the previous example), even
+though their access by the topology is still through a file called
+*blob_file*.
+
+### Removing a cached file
+
+To remove a file from the distributed cache, use the following command. Removing
+a file requires write access.
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+storm blobstore delete [KEYSTRING]
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+### Listing Blobs currently in the distributed cache blob store
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+storm blobstore list [KEY...]
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+lists blobs currently in the blob store
+
+### Reading the contents of a blob
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+storm blobstore cat [-f|--file FILE] KEY
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+read a blob and then either write it to a file, or STDOUT. Reading a blob
+requires read access.
+
+### Setting the access control for a blob
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+set-acl [-s ACL] KEY
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+ACL is in the form [uo]:[username]:[r-][w-][a-] can be comma  separated list
+(requires admin access).
+
+### Update the replication factor for a blob
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+storm blobstore replication --update --repl-fctr 5 key1
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+### Read the replication factor of a blob
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+storm blobstore replication --read key1
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+### Command line help
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+storm help blobstore
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+## Using the Distributed Cache API from Java
+
+We start by getting a ClientBlobStore object by calling this function:
+
+``` java
+Config theconf = new Config();
+theconf.putAll(Utils.readStormConfig());
+ClientBlobStore clientBlobStore = Utils.getClientBlobStore(theconf);
+```
+
+The required Utils package can by imported by:
+
+```java
+import backtype.storm.utils.Utils;
+```
+
+ClientBlobStore and other blob-related classes can be imported by:
+
+```java
+import backtype.storm.blobstore.ClientBlobStore;
+import backtype.storm.blobstore.AtomicOutputStream;
+import backtype.storm.blobstore.InputStreamWithMeta;
+import backtype.storm.blobstore.BlobStoreAclHandler;
+import backtype.storm.generated.*;
+```
+
+### Creating ACLs to be used for blobs
+
+```java
+String stringBlobACL = "u:username:rwa";
+AccessControl blobACL = BlobStoreAclHandler.parseAccessControl(stringBlobACL);
+List<AccessControl> acls = new LinkedList<AccessControl>();
+acls.add(blobACL); // more ACLs can be added here
+SettableBlobMeta settableBlobMeta = new SettableBlobMeta(acls);
+settableBlobMeta.set_replication_factor(4); // Here we can set the replication factor
+```
+
+The settableBlobMeta object is what we need to create a blob in the next step. 
+
+### Creating a blob
+
+```java
+AtomicOutputStream blobStream = clientBlobStore.createBlob("some_key", settableBlobMeta);
+blobStream.write("Some String or input data".getBytes());
+blobStream.close();
+```
+
+Note that the settableBlobMeta object here comes from the last step, creating ACLs.
+It is recommended that for very large files, the user writes the bytes in smaller chunks (for example 64 KB, up to 1 MB chunks).
+
+### Updating a blob
+
+Similar to creating a blob, but we get the AtomicOutputStream in a different way:
+
+```java
+String blobKey = "some_key";
+AtomicOutputStream blobStream = clientBlobStore.updateBlob(blobKey);
+```
+
+Pass a byte stream to the returned AtomicOutputStream as before. 
+
+### Updating the ACLs of a blob
+
+```java
+String blobKey = "some_key";
+AccessControl updateAcl = BlobStoreAclHandler.parseAccessControl("u:USER:--a");
+List<AccessControl> updateAcls = new LinkedList<AccessControl>();
+updateAcls.add(updateAcl);
+SettableBlobMeta modifiedSettableBlobMeta = new SettableBlobMeta(updateAcls);
+clientBlobStore.setBlobMeta(blobKey, modifiedSettableBlobMeta);
+
+//Now set write only
+updateAcl = BlobStoreAclHandler.parseAccessControl("u:USER:-w-");
+updateAcls = new LinkedList<AccessControl>();
+updateAcls.add(updateAcl);
+modifiedSettableBlobMeta = new SettableBlobMeta(updateAcls);
+clientBlobStore.setBlobMeta(blobKey, modifiedSettableBlobMeta);
+```
+
+### Updating and Reading the replication of a blob
+
+```java
+String blobKey = "some_key";
+BlobReplication replication = clientBlobStore.updateBlobReplication(blobKey, 5);
+int replication_factor = replication.get_replication();
+```
+
+Note: The replication factor gets updated and reflected only for hdfs blobstore
+
+### Reading a blob
+
+```java
+String blobKey = "some_key";
+InputStreamWithMeta blobInputStream = clientBlobStore.getBlob(blobKey);
+BufferedReader r = new BufferedReader(new InputStreamReader(blobInputStream));
+String blobContents =  r.readLine();
+```
+
+### Deleting a blob
+
+```java
+String blobKey = "some_key";
+clientBlobStore.deleteBlob(blobKey);
+```
+
+### Getting a list of blob keys already in the blobstore
+
+```java
+Iterator <String> stringIterator = clientBlobStore.listKeys();
+```
+
+## Appendix A
+
+```java
+public abstract void prepare(Map conf, String baseDir);
+
+public abstract AtomicOutputStream createBlob(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException;
+
+public abstract AtomicOutputStream updateBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
+
+public abstract ReadableBlobMeta getBlobMeta(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
+
+public abstract void setBlobMeta(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyNotFoundException;
+
+public abstract void deleteBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
+
+public abstract InputStreamWithMeta getBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
+
+public abstract Iterator<String> listKeys(Subject who);
+
+public abstract BlobReplication getBlobReplication(String key, Subject who) throws Exception;
+
+public abstract BlobReplication updateBlobReplication(String key, int replication, Subject who) throws AuthorizationException, KeyNotFoundException, IOException
+```
+
+## Appendix B
+
+```java
+public abstract void prepare(Map conf);
+
+protected abstract AtomicOutputStream createBlobToExtend(String key, SettableBlobMeta meta) throws AuthorizationException, KeyAlreadyExistsException;
+
+public abstract AtomicOutputStream updateBlob(String key) throws AuthorizationException, KeyNotFoundException;
+
+public abstract ReadableBlobMeta getBlobMeta(String key) throws AuthorizationException, KeyNotFoundException;
+
+protected abstract void setBlobMetaToExtend(String key, SettableBlobMeta meta) throws AuthorizationException, KeyNotFoundException;
+
+public abstract void deleteBlob(String key) throws AuthorizationException, KeyNotFoundException;
+
+public abstract InputStreamWithMeta getBlob(String key) throws AuthorizationException, KeyNotFoundException;
+
+public abstract Iterator<String> listKeys();
+
+public abstract void watchBlob(String key, IBlobWatcher watcher) throws AuthorizationException;
+
+public abstract void stopWatchingBlob(String key) throws AuthorizationException;
+
+public abstract BlobReplication getBlobReplication(String Key) throws AuthorizationException, KeyNotFoundException;
+
+public abstract BlobReplication updateBlobReplication(String Key, int replication) throws AuthorizationException, KeyNotFoundException
+```
+
+## Appendix C
+
+``` thrift
+service Nimbus {
+...
+string beginCreateBlob(1: string key, 2: SettableBlobMeta meta) throws (1: AuthorizationException aze, 2: KeyAlreadyExistsException kae);
+
+string beginUpdateBlob(1: string key) throws (1: AuthorizationException aze, 2: KeyNotFoundException knf);
+
+void uploadBlobChunk(1: string session, 2: binary chunk) throws (1: AuthorizationException aze);
+
+void finishBlobUpload(1: string session) throws (1: AuthorizationException aze);
+
+void cancelBlobUpload(1: string session) throws (1: AuthorizationException aze);
+
+ReadableBlobMeta getBlobMeta(1: string key) throws (1: AuthorizationException aze, 2: KeyNotFoundException knf);
+
+void setBlobMeta(1: string key, 2: SettableBlobMeta meta) throws (1: AuthorizationException aze, 2: KeyNotFoundException knf);
+
+BeginDownloadResult beginBlobDownload(1: string key) throws (1: AuthorizationException aze, 2: KeyNotFoundException knf);
+
+binary downloadBlobChunk(1: string session) throws (1: AuthorizationException aze);
+
+void deleteBlob(1: string key) throws (1: AuthorizationException aze, 2: KeyNotFoundException knf);
+
+ListBlobsResult listBlobs(1: string session);
+
+BlobReplication getBlobReplication(1: string key) throws (1: AuthorizationException aze, 2: KeyNotFoundException knf);
+
+BlobReplication updateBlobReplication(1: string key, 2: i32 replication) throws (1: AuthorizationException aze, 2: KeyNotFoundException knf);
+...
+}
+
+struct BlobReplication {
+1: required i32 replication;
+}
+
+exception AuthorizationException {
+ 1: required string msg;
+}
+
+exception KeyNotFoundException {
+ 1: required string msg;
+}
+
+exception KeyAlreadyExistsException {
+ 1: required string msg;
+}
+
+enum AccessControlType {
+ OTHER = 1,
+ USER = 2
+ //eventually ,GROUP=3
+}
+
+struct AccessControl {
+ 1: required AccessControlType type;
+ 2: optional string name; //Name of user or group in ACL
+ 3: required i32 access; //bitmasks READ=0x1, WRITE=0x2, ADMIN=0x4
+}
+
+struct SettableBlobMeta {
+ 1: required list<AccessControl> acl;
+ 2: optional i32 replication_factor
+}
+
+struct ReadableBlobMeta {
+ 1: required SettableBlobMeta settable;
+ //This is some indication of a version of a BLOB.  The only guarantee is
+ // if the data changed in the blob the version will be different.
+ 2: required i64 version;
+}
+
+struct ListBlobsResult {
+ 1: required list<string> keys;
+ 2: required string session;
+}
+
+struct BeginDownloadResult {
+ //Same version as in ReadableBlobMeta
+ 1: required i64 version;
+ 2: required string session;
+ 3: optional i64 data_size;
+}
+```

http://git-wip-us.apache.org/repos/asf/storm/blob/63b2e3d3/docs/documentation/images/hdfs_blobstore.png
----------------------------------------------------------------------
diff --git a/docs/documentation/images/hdfs_blobstore.png b/docs/documentation/images/hdfs_blobstore.png
new file mode 100644
index 0000000..11c5c10
Binary files /dev/null and b/docs/documentation/images/hdfs_blobstore.png differ

http://git-wip-us.apache.org/repos/asf/storm/blob/63b2e3d3/docs/documentation/images/local_blobstore.png
----------------------------------------------------------------------
diff --git a/docs/documentation/images/local_blobstore.png b/docs/documentation/images/local_blobstore.png
new file mode 100644
index 0000000..ff8001e
Binary files /dev/null and b/docs/documentation/images/local_blobstore.png differ

http://git-wip-us.apache.org/repos/asf/storm/blob/63b2e3d3/docs/documentation/images/nimbus_ha_blobstore.png
----------------------------------------------------------------------
diff --git a/docs/documentation/images/nimbus_ha_blobstore.png b/docs/documentation/images/nimbus_ha_blobstore.png
new file mode 100644
index 0000000..26e8c2a
Binary files /dev/null and b/docs/documentation/images/nimbus_ha_blobstore.png differ


[2/6] storm git commit: fixing concerns related to formatting and spellings

Posted by kn...@apache.org.
fixing concerns related to formatting and spellings


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

Branch: refs/heads/master
Commit: a9a84206cd9a86e5253b99d66e5f30ce33ffe3a8
Parents: 63b2e3d
Author: Sanket <sc...@untilservice-lm>
Authored: Mon Dec 14 16:34:10 2015 -0600
Committer: Sanket <sc...@untilservice-lm>
Committed: Mon Dec 14 16:35:56 2015 -0600

----------------------------------------------------------------------
 docs/documentation/distcache-blobstore.md | 62 ++++++++++++++------------
 1 file changed, 33 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/a9a84206/docs/documentation/distcache-blobstore.md
----------------------------------------------------------------------
diff --git a/docs/documentation/distcache-blobstore.md b/docs/documentation/distcache-blobstore.md
index 2de14bb..87265a1 100644
--- a/docs/documentation/distcache-blobstore.md
+++ b/docs/documentation/distcache-blobstore.md
@@ -36,7 +36,7 @@ The current BlobStore interface has the following two implementations
 * LocalFsBlobStore
 * HdfsBlobStore
 
-Appendix A contains the interface for blob store implementation.
+Appendix A contains the interface for blobstore implementation.
 
 ## LocalFsBlobStore
 ![LocalFsBlobStore](images/local_blobstore.png)
@@ -48,7 +48,11 @@ The main stages can be depicted as follows
 
 ### Blob Creation Command
 Blobs in the blobstore can be created through command line using the following command.
+
+```
 storm blobstore create --file README.txt --acl o::rwa --repl-fctr 4 key1
+```
+
 The above command creates a blob with a key name “key1” corresponding to the file README.txt. 
 The access given to all users being read, write and admin with a replication factor of 4.
 
@@ -66,7 +70,7 @@ storm.starter.clj.word_count test_topo -c topology.blobstore.map='{"key1":{"loca
 The creation of the blob takes place through the interface “ClientBlobStore”. Appendix B contains the “ClientBlobStore” interface. 
 The concrete implementation of this interface is the  “NimbusBlobStore”. In the case of local file system the client makes a 
 call to the nimbus to create the blobs within the local file system. The nimbus uses the local file system implementation to create these blobs. 
-When a user submits a topology, the jar, configuration and code files are uploaded as blobs with the help of blob store. 
+When a user submits a topology, the jar, configuration and code files are uploaded as blobs with the help of blobstore. 
 Also, all the other blobs specified by the topology are mapped to it with the help of topology.blobstore.map configuration.
 
 ### Blob Download by the Supervisor
@@ -81,10 +85,10 @@ the topologies.
 ![HdfsBlobStore](images/hdfs_blobstore.png)
 
 The HdfsBlobStore functionality has a similar implementation and blob creation and download procedure barring how the replication 
-is handled in the two blob store implementations. The replication in HDFS blob store is obvious as HDFS is equipped to handle replication 
+is handled in the two blobstore implementations. The replication in HDFS blobstore is obvious as HDFS is equipped to handle replication 
 and it requires no state to be stored inside the zookeeper. On the other hand, the local file system blobstore requires the state to be 
 stored on the zookeeper in order for it to work with nimbus HA. Nimbus HA allows the local filesystem to implement the replication feature 
-seamlessly by storing the state in the zookeeper about the running topologies and syncing the blobs on various nimbodes. On the supervisor’s 
+seamlessly by storing the state in the zookeeper about the running topologies and syncing the blobs on various nimbuses. On the supervisor’s 
 end, the supervisor and localizer talks to HdfsBlobStore through “HdfsClientBlobStore” implementation.
 
 ## Additional Features and Documentation
@@ -94,9 +98,9 @@ storm jar /home/y/lib/storm-starter/current/storm-starter-jar-with-dependencies.
 ```
  
 ### Compression
-The blob store allows the user to specify the “uncompress” configuration to true or false. This configuration can be specified 
+The blobstore allows the user to specify the “uncompress” configuration to true or false. This configuration can be specified 
 in the topology.blobstore.map mentioned in the above command. This allows the user to upload a compressed file like a tarball/zip. 
-In local file system blob store, the compressed blobs are stored on the nimbus node. The localizer code takes the responsibility to 
+In local file system blobstore, the compressed blobs are stored on the nimbus node. The localizer code takes the responsibility to 
 uncompress the blob and store it on the supervisor node. Symbolic links to the blobs on the supervisor node are created within the worker 
 before the execution starts.
 
@@ -104,12 +108,12 @@ before the execution starts.
 Apart from compression the blobstore helps to give the blob a name that can be used by the workers. The localizer takes 
 the responsibility of mapping the blob to a local name on the supervisor node.
 
-## Additional Blob Store Implementation Details
-Blob store uses a hashing function to create the blobs based on the key. The blobs are generally stored inside the directory specified by 
+## Additional blobstore Implementation Details
+blobstore uses a hashing function to create the blobs based on the key. The blobs are generally stored inside the directory specified by 
 the blobstore.dir configuration. By default, it is stored under “storm.local.dir/nimbus/blobs” for local file system and a similar path on 
 hdfs file system.
 
-Once a file is submitted, the blob store reads the configs and creates a metadata for the blob with all the access control details. The metadata 
+Once a file is submitted, the blobstore reads the configs and creates a metadata for the blob with all the access control details. The metadata 
 is generally used for authorization while accessing the blobs. The blob key and version contribute to the hash code and there by the directory 
 under “storm.local.dir/nimbus/blobs/data” where the data is placed. The blobs are generally placed in a positive number directory like 193,822 etc.
 
@@ -121,14 +125,14 @@ This allows updating the blobs on the fly and thereby making it a very useful fe
 For a local file system, the distributed cache on the supervisor node is set to 10240 MB as a soft limit and the clean up code attempts 
 to clean anything over the soft limit every 600 seconds based on LRU policy.
 
-The HDFS blob store implementation handles load better by removing the burden on the nimbus to store the blobs, which avoids it becoming a bottleneck. Moreover, it provides seamless replication of blobs. On the other hand, the local file system blob store is not very efficient in 
-replicating the blobs and is limited by the number of nimbuses. Moreover, the supervisor talks to the HDFS blob store directly without the 
+The HDFS blobstore implementation handles load better by removing the burden on the nimbus to store the blobs, which avoids it becoming a bottleneck. Moreover, it provides seamless replication of blobs. On the other hand, the local file system blobstore is not very efficient in 
+replicating the blobs and is limited by the number of nimbuses. Moreover, the supervisor talks to the HDFS blobstore directly without the 
 involvement of the nimbus and thereby reduces the load and dependency on nimbus.
 
 ## Highly Available Nimbus
 ### Problem Statement:
 Currently the storm master aka nimbus, is a process that runs on a single machine under supervision. In most cases the 
-nimbus failure is transient and it is restarted by the supervisor. However sometimes when disks fail and networks 
+nimbus failure is transient and it is restarted by the process that does supervision. However sometimes when disks fail and networks 
 partitions occur, nimbus goes down. Under these circumstances the topologies run normally but no new topologies can be 
 submitted, no existing topologies can be killed/deactivated/activated and if a supervisor node fails then the 
 reassignments are not performed resulting in performance degradation or topology failures. With this project we intend 
@@ -183,7 +187,7 @@ If the topology code, jar or config blobs are missing, it would download the blo
 
 The first implementation will be Zookeeper based. If the zookeeper connection is lost/reset resulting in loss of lock
 or the spot in queue the implementation will take care of updating the state such that isLeader() will reflect the 
-current status.The leader like actions must finish in less than minimumOf(connectionTimeout, SessionTimeout) to ensure
+current status. The leader like actions must finish in less than minimumOf(connectionTimeout, SessionTimeout) to ensure
 the lock was held by nimbus for the entire duration of the action (Not sure if we want to just state this expectation 
 and ensure that zk configurations are set high enough which will result in higher failover time or we actually want to 
 create some sort of rollback mechanism for all actions, the second option needs a lot of code). If a nimbus that is not 
@@ -194,7 +198,7 @@ leader receives a request that only a leader can perform it will throw a RunTime
 To achieve fail over from primary to backup servers nimbus state/data needs to be replicated across all nimbus hosts or 
 needs to be stored in a distributed storage. Replicating the data correctly involves state management, consistency checks
 and it is hard to test for correctness. However many storm users do not want to take extra dependency on another replicated
-storage system like HDFS and still need high availability. The blob store implementation along with the state storage helps
+storage system like HDFS and still need high availability. The blobstore implementation along with the state storage helps
 to overcome the failover scenarios in case a leader nimbus goes down.
 
 To support replication we will allow the user to define a code replication factor which would reflect number of nimbus 
@@ -206,11 +210,11 @@ tries to download them as and when they are needed. With this current architectu
 required for a topology for a nimbus to accept leadership. This helps us in case the blobs are very large and avoid causing any 
 inadvertant delays in electing a leader.
 
-The state for every blob is relevant for the local blob store implementation. For HDFS blob store the replication
-is taken care by the HDFS. For handling the fail over scenarios for a local blob store we need to store the state of the leader and
-non-leader nimbodes within the zookeeper.
+The state for every blob is relevant for the local blobstore implementation. For HDFS blobstore the replication
+is taken care by the HDFS. For handling the fail over scenarios for a local blobstore we need to store the state of the leader and
+non-leader nimbuses within the zookeeper.
 
-The state is stored under /storm/blobstore/key/nimbusHostPort:SequenceNumber for the blob store to work to make nimbus highly available. 
+The state is stored under /storm/blobstore/key/nimbusHostPort:SequenceNumber for the blobstore to work to make nimbus highly available. 
 This state is used in the local file system blobstore to support replication. The HDFS blobstore does not have to store the state inside the 
 zookeeper.
 
@@ -223,9 +227,9 @@ stored under /storm/blobstoremaxsequencenumber/key. For more details about how t
 
 ![Nimbus High Availability - BlobStore](images/nimbus_ha_blobstore.png)
 
-The sequence diagram proposes how the blob store works and the state storage inside the zookeeper makes the nimbus highly available.
+The sequence diagram proposes how the blobstore works and the state storage inside the zookeeper makes the nimbus highly available.
 Currently, the thread to sync the blobs on a non-leader is within the nimbus. In the future, it will be nice to move the thread around
-to the blob store to make the blobstore coordinate the state change and blob download as per the sequence diagram.
+to the blobstore to make the blobstore coordinate the state change and blob download as per the sequence diagram.
 
 ## Thrift and Rest API 
 In order to avoid workers/supervisors/ui talking to zookeeper for getting master nimbus address we are going to modify the 
@@ -256,7 +260,7 @@ be rare in general case.
 
 Note: All nimbus hosts have watchers on zookeeper to be notified immediately as soon as a new blobs is available for download, the callback may or may not download
 the code. Therefore, a background thread is triggered to download the respective blobs to run the topologies. The replication is achieved when the blobs are downloaded
-onto non-leader nimbodes. So you should expect your topology submission time to be somewhere between 0 to (2 * nimbus.code.sync.freq.secs) for any 
+onto non-leader nimbuses. So you should expect your topology submission time to be somewhere between 0 to (2 * nimbus.code.sync.freq.secs) for any 
 nimbus.min.replication.count > 1.
 
 ## Configuration
@@ -265,8 +269,8 @@ nimbus.min.replication.count > 1.
 blobstore.dir: The directory where all blobs are stored. For local file system it represents the directory on the nimbus 
 node and for HDFS file system it represents the hdfs file system path.
 
-supervisor.blobstore.class: This configuration is meant to set the client for  the supervisor  in order to talk to the blob store. 
-For a local file system blob store it is set to “backtype.storm.blobstore.NimbusBlobStore” and for the HDFS blob store it is set 
+supervisor.blobstore.class: This configuration is meant to set the client for  the supervisor  in order to talk to the blobstore. 
+For a local file system blobstore it is set to “backtype.storm.blobstore.NimbusBlobStore” and for the HDFS blobstore it is set 
 to “backtype.storm.blobstore.HdfsClientBlobStore”.
 
 supervisor.blobstore.download.thread.count: This configuration spawns multiple threads for from the supervisor in order download 
@@ -285,15 +289,15 @@ cleanup anything over the cache target size. By default it is set to 600000 mill
 
 nimbus.blobstore.class:  Sets the blobstore implementation nimbus uses. It is set to "backtype.storm.blobstore.LocalFsBlobStore"
 
-nimbus.blobstore.expiration.secs: During operations with the blob store, via master, how long a connection is idle before nimbus 
+nimbus.blobstore.expiration.secs: During operations with the blobstore, via master, how long a connection is idle before nimbus 
 considers it dead and drops the session and any associated connections. The default is set to 600.
 
-storm.blobstore.inputstream.buffer.size.bytes: The buffer size it uses for blob store upload. It is set to 65536 bytes.
+storm.blobstore.inputstream.buffer.size.bytes: The buffer size it uses for blobstore upload. It is set to 65536 bytes.
 
-client.blobstore.class: The blob store implementation the storm client uses. The current implementation uses the default 
+client.blobstore.class: The blobstore implementation the storm client uses. The current implementation uses the default 
 config "backtype.storm.blobstore.NimbusBlobStore".
 
-blobstore.replication.factor: It sets the replication for each blob within the blob store. The “topology.min.replication.count” 
+blobstore.replication.factor: It sets the replication for each blob within the blobstore. The “topology.min.replication.count” 
 ensures the minimum replication the topology specific blobs are set before launching the topology. You might want to set the 
 “topology.min.replication.count <= blobstore.replication”. The default is set to 3.
 
@@ -438,13 +442,13 @@ a file requires write access.
 storm blobstore delete [KEYSTRING]
 ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
 
-### Listing Blobs currently in the distributed cache blob store
+### Listing Blobs currently in the distributed cache blobstore
 
 ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
 storm blobstore list [KEY...]
 ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
 
-lists blobs currently in the blob store
+lists blobs currently in the blobstore
 
 ### Reading the contents of a blob
 


[5/6] storm git commit: Merge branch 'distcache-docs' of https://github.com/redsanket/storm

Posted by kn...@apache.org.
Merge branch 'distcache-docs' of https://github.com/redsanket/storm


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

Branch: refs/heads/master
Commit: 40823ecef65d6774c2d0f22e9fcdad956e8190a6
Parents: 6cf33a8 eb30b81
Author: Kyle Nusbaum <Ky...@gmail.com>
Authored: Thu Dec 17 14:45:57 2015 -0600
Committer: Kyle Nusbaum <Ky...@gmail.com>
Committed: Thu Dec 17 14:45:57 2015 -0600

----------------------------------------------------------------------
 docs/documentation/distcache-blobstore.md       | 735 +++++++++++++++++++
 docs/documentation/images/hdfs_blobstore.png    | Bin 0 -> 82180 bytes
 docs/documentation/images/local_blobstore.png   | Bin 0 -> 81212 bytes
 .../images/nimbus_ha_blobstore.png              | Bin 0 -> 113991 bytes
 docs/documentation/nimbus-ha-design.md          | 217 ------
 5 files changed, 735 insertions(+), 217 deletions(-)
----------------------------------------------------------------------



[6/6] storm git commit: Adding STORM-1372 to CHANGELOG

Posted by kn...@apache.org.
Adding STORM-1372 to CHANGELOG


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

Branch: refs/heads/master
Commit: 0e1c969c20bc3039935fb4af6582702b726e5162
Parents: 40823ec
Author: Kyle Nusbaum <Ky...@gmail.com>
Authored: Thu Dec 17 14:47:10 2015 -0600
Committer: Kyle Nusbaum <Ky...@gmail.com>
Committed: Thu Dec 17 14:47:10 2015 -0600

----------------------------------------------------------------------
 CHANGELOG.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/0e1c969c/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 96e8e41..d74ca54 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 ## 0.11.0
+ * STORM-1372: Merging design and usage documents for distcache
  * STORM-1393: Update the storm.log.dir function, add doc for logs
  * STORM-1377: nimbus_auth_test: very short timeouts causing spurious failures
  * STORM-1388: Fix url and email links in README file


[4/6] storm git commit: capitalization and quote consistency change

Posted by kn...@apache.org.
capitalization and quote consistency change


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

Branch: refs/heads/master
Commit: eb30b81c76ea51793e1a2b4ba19e76b87caa3428
Parents: ea3aa2e
Author: Sanket <sc...@untilservice-lm>
Authored: Wed Dec 16 11:14:05 2015 -0600
Committer: Sanket <sc...@untilservice-lm>
Committed: Wed Dec 16 11:14:05 2015 -0600

----------------------------------------------------------------------
 docs/documentation/distcache-blobstore.md | 11 +++++------
 1 file changed, 5 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/eb30b81c/docs/documentation/distcache-blobstore.md
----------------------------------------------------------------------
diff --git a/docs/documentation/distcache-blobstore.md b/docs/documentation/distcache-blobstore.md
index 13d3732..2011ce3 100644
--- a/docs/documentation/distcache-blobstore.md
+++ b/docs/documentation/distcache-blobstore.md
@@ -108,8 +108,8 @@ before the execution starts.
 Apart from compression the blobstore helps to give the blob a name that can be used by the workers. The localizer takes 
 the responsibility of mapping the blob to a local name on the supervisor node.
 
-## Additional blobstore Implementation Details
-blobstore uses a hashing function to create the blobs based on the key. The blobs are generally stored inside the directory specified by 
+## Additional Blobstore Implementation Details
+Blobstore uses a hashing function to create the blobs based on the key. The blobs are generally stored inside the directory specified by
 the blobstore.dir configuration. By default, it is stored under “storm.local.dir/nimbus/blobs” for local file system and a similar path on 
 hdfs file system.
 
@@ -265,8 +265,8 @@ nimbus.min.replication.count > 1.
 
 ## Configuration
 
-~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
-blobstore.dir: The directory where all blobs are stored. For local file system it represents the directory on the nimbus 
+```
+blobstore.dir: The directory where all blobs are stored. For local file system it represents the directory on the nimbus
 node and for HDFS file system it represents the hdfs file system path.
 
 supervisor.blobstore.class: This configuration is meant to set the client for  the supervisor  in order to talk to the blobstore. 
@@ -308,8 +308,7 @@ topology.max.replication.wait.time.sec: Maximum wait time for the nimbus host re
 Once this time is elapsed nimbus will go ahead and perform topology activation tasks even if required nimbus.min.replication.count is not achieved. 
 The default is 60 seconds, a value of -1 indicates to wait for ever.
 * nimbus.code.sync.freq.secs: Frequency at which the background thread on nimbus which syncs code for locally missing blobs. Default is 2 minutes.
-~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
-
+```
 
 ## Using the Distributed Cache API, Command Line Interface (CLI)
 


[3/6] storm git commit: comma corrections and removed nimbus-ha design doc as it is now merged with blobstore

Posted by kn...@apache.org.
comma corrections and removed nimbus-ha design doc as it is now merged with blobstore

added comma


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

Branch: refs/heads/master
Commit: ea3aa2e3802e19580974899b51368b5eb2850116
Parents: a9a8420
Author: Sanket <sc...@untilservice-lm>
Authored: Tue Dec 15 20:33:41 2015 -0600
Committer: Sanket <sc...@untilservice-lm>
Committed: Tue Dec 15 20:39:25 2015 -0600

----------------------------------------------------------------------
 docs/documentation/distcache-blobstore.md |  10 +-
 docs/documentation/nimbus-ha-design.md    | 217 -------------------------
 2 files changed, 5 insertions(+), 222 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/ea3aa2e3/docs/documentation/distcache-blobstore.md
----------------------------------------------------------------------
diff --git a/docs/documentation/distcache-blobstore.md b/docs/documentation/distcache-blobstore.md
index 87265a1..13d3732 100644
--- a/docs/documentation/distcache-blobstore.md
+++ b/docs/documentation/distcache-blobstore.md
@@ -117,7 +117,7 @@ Once a file is submitted, the blobstore reads the configs and creates a metadata
 is generally used for authorization while accessing the blobs. The blob key and version contribute to the hash code and there by the directory 
 under “storm.local.dir/nimbus/blobs/data” where the data is placed. The blobs are generally placed in a positive number directory like 193,822 etc.
 
-Once the topology is launched and the relevant blobs have been created the supervisor downloads blobs related to the storm.conf, storm.ser 
+Once the topology is launched and the relevant blobs have been created, the supervisor downloads blobs related to the storm.conf, storm.ser 
 and storm.code first and all the blobs uploaded by the command line separately using the localizer to uncompress and map them to a local name 
 specified in the topology.blobstore.map configuration. The supervisor periodically updates blobs by checking for the change of version. 
 This allows updating the blobs on the fly and thereby making it a very useful feature.
@@ -131,11 +131,11 @@ involvement of the nimbus and thereby reduces the load and dependency on nimbus.
 
 ## Highly Available Nimbus
 ### Problem Statement:
-Currently the storm master aka nimbus, is a process that runs on a single machine under supervision. In most cases the 
+Currently the storm master aka nimbus, is a process that runs on a single machine under supervision. In most cases, the 
 nimbus failure is transient and it is restarted by the process that does supervision. However sometimes when disks fail and networks 
-partitions occur, nimbus goes down. Under these circumstances the topologies run normally but no new topologies can be 
+partitions occur, nimbus goes down. Under these circumstances, the topologies run normally but no new topologies can be 
 submitted, no existing topologies can be killed/deactivated/activated and if a supervisor node fails then the 
-reassignments are not performed resulting in performance degradation or topology failures. With this project we intend 
+reassignments are not performed resulting in performance degradation or topology failures. With this project we intend, 
 to resolve this problem by running nimbus in a primary backup mode to guarantee that even if a nimbus server fails one 
 of the backups will take over. 
 
@@ -191,7 +191,7 @@ current status. The leader like actions must finish in less than minimumOf(conne
 the lock was held by nimbus for the entire duration of the action (Not sure if we want to just state this expectation 
 and ensure that zk configurations are set high enough which will result in higher failover time or we actually want to 
 create some sort of rollback mechanism for all actions, the second option needs a lot of code). If a nimbus that is not 
-leader receives a request that only a leader can perform it will throw a RunTimeException.
+leader receives a request that only a leader can perform,  it will throw a RunTimeException.
 
 ### Nimbus state store:
 

http://git-wip-us.apache.org/repos/asf/storm/blob/ea3aa2e3/docs/documentation/nimbus-ha-design.md
----------------------------------------------------------------------
diff --git a/docs/documentation/nimbus-ha-design.md b/docs/documentation/nimbus-ha-design.md
deleted file mode 100644
index 672eece..0000000
--- a/docs/documentation/nimbus-ha-design.md
+++ /dev/null
@@ -1,217 +0,0 @@
-#Highly Available Nimbus design proposal
-##Problem Statement:
-Currently the storm master aka nimbus, is a process that runs on a single machine under supervision. In most cases the 
-nimbus failure is transient and it is restarted by the supervisor. However sometimes when disks fail and networks 
-partitions occur, nimbus goes down. Under these circumstances the topologies run normally but no new topologies can be 
-submitted, no existing topologies can be killed/deactivated/activated and if a supervisor node fails then the 
-reassignments are not performed resulting in performance degradation or topology failures. With this project we intend 
-to resolve this problem by running nimbus in a primary backup mode to guarantee that even if a nimbus server fails one 
-of the backups will take over.
-##Requirements:
-* Increase overall availability of nimbus.
-* Allow nimbus hosts to leave and join the cluster at will any time. A newly joined host should auto catch up and join 
-the list of potential leaders automatically. 
-* No topology resubmissions required in case of nimbus fail overs.
-* No active topology should ever be lost. 
-
-##Leader Election:
-The nimbus server will use the following interface:
-
-```java
-public interface ILeaderElector {
-    /**
-     * queue up for leadership lock. The call returns immediately and the caller                     
-     * must check isLeader() to perform any leadership action.
-     */
-    void addToLeaderLockQueue();
-
-    /**
-     * Removes the caller from the leader lock queue. If the caller is leader
-     * also releases the lock.
-     */
-    void removeFromLeaderLockQueue();
-
-    /**
-     *
-     * @return true if the caller currently has the leader lock.
-     */
-    boolean isLeader();
-
-    /**
-     *
-     * @return the current leader's address , throws exception if noone has has    lock.
-     */
-    InetSocketAddress getLeaderAddress();
-
-    /**
-     * 
-     * @return list of current nimbus addresses, includes leader.
-     */
-    List<InetSocketAddress> getAllNimbusAddresses();
-}
-```
-On startup nimbus will check if it has code for all active topologies available locally. Once it gets to this state it 
-will call addToLeaderLockQueue() function. When a nimbus is notified to become a leader it will check if it has all the
-code locally before assuming the leadership role. If any active topology code is missing, the node will not accept the 
-leadership role instead it will release the lock and wait till it has all the code before requeueing for leader lock. 
-
-The first implementation will be Zookeeper based. If the zookeeper connection is lost/resetted resulting in loss of lock
-or the spot in queue the implementation will take care of updating the state such that isLeader() will reflect the 
-current status.The leader like actions must finish in less than minimumOf(connectionTimeout, SessionTimeout) to ensure
-the lock was held by nimbus for the entire duration of the action (Not sure if we want to just state this expectation 
-and ensure that zk configurations are set high enough which will result in higher failover time or we actually want to 
-create some sort of rollback mechanism for all actions, the second option needs a lot of code). If a nimbus that is not 
-leader receives a request that only a leader can perform it will throw a RunTimeException.
-
-Following steps describes a nimbus failover scenario:
-* Let’s say we have 4 topologies running with 3 nimbus nodes and code-replication-factor = 2. We assume that the 
-invariant “The leader nimbus has code for all topologies locally” holds true at the beginning. nonleader-1 has code for 
-the first 2 topologies and nonLeader-2 has code for the other 2 topologies.
-* Leader nimbus dies, hard disk failure so no recovery possible.
-* nonLeader-1 gets a zookeeper notification to indicate it is now the new leader. before accepting the leadership it 
-checks if it has code available for all 4 topologies(these are topologies under /storm/storms/). It realizes it only has
-code for 2 topologies so it relinquishes the lock and looks under  /storm/code-distributor/topologyId to find out from 
-where can it download the code/metafile for the missing topologies. it finds entries for the leader nimbus and 
-nonleader-2. It will try downloading from both as part of its retry mechanism.
-* nonLeader-2’s code sync thread also realizes that it is missing code for 2 topologies and follows the same process 
-described in step-3 to download code for missing topologies. 
-* eventually at least one of the nimbuses will have all the code locally and will accept leadership.
-This sequence diagram describes how leader election and failover would work with multiple components.
-
-![Nimbus Fail Over](images/nimbus_ha_leader_election_and_failover.png)
-
-##Nimbus state store:
-
-Currently the nimbus stores 2 kind of data
-* Meta information like supervisor info, assignment info which is stored in zookeeper
-* Actual topology configs and jars that is stored on nimbus host’s local disk.
-
-To achieve fail over from primary to backup servers nimbus state/data needs to be replicated across all nimbus hosts or 
-needs to be stored in a distributed storage. Replicating the data correctly involves state management, consistency checks
-and it is hard to test for correctness.However many storm users do not want to take extra dependency on another replicated
-storage system like HDFS and still need high availability.Eventually, we want to move to the bittorrent protocol for code 
-distribution given the size of the jars and to achieve better scaling when the total number of supervisors is very high. 
-The current file system based model for code distribution works fine with systems that have file system like structure
-but it fails to support a non file system based approach like bit torrent. To support bit torrent and all the file
-system based replicated storage systems we propose the following interface:
-
-```java
-/**
- * Interface responsible to distribute code in the cluster.
- */
-public interface ICodeDistributor {
-    /**
-     * Prepare this code distributor.
-     * @param conf
-     */
-    void prepare(Map conf);
-
-    /**
-     * This API will perform the actual upload of the code to the distributed implementation.
-     * The API should return a Meta file which should have enough information for downloader 
-     * so it can download the code e.g. for bittorrent it will be a torrent file, in case of something         
-     * like HDFS or s3  it might have the actual directory or paths for files to be downloaded.
-     * @param dirPath local directory where all the code to be distributed exists.
-     * @param topologyId the topologyId for which the meta file needs to be created.
-     * @return metaFile
-     */
-    File upload(Path dirPath, String topologyId);
-
-    /**
-     * Given the topologyId and metafile, download the actual code and return the downloaded file's list.
-     * @param topologyid
-     * @param metafile 
-     * @param destDirPath the folder where all the files will be downloaded.
-     * @return
-     */
-    List<File> download(Path destDirPath, String topologyid, File metafile);
-
-    /**
-      * Given the topologyId, returns number of hosts where the code has been replicated.
-      */
-    int getReplicationCount(String topologyId);
-    
-   /**
-     * Performs the cleanup.
-     * @param topologyid
-     */
-    void cleanup(String topologyid);
-
-    /**
-     * Close this distributor.
-     * @param conf
-     */
-    void close(Map conf);
-}
-```
-To support replication we will allow the user to define a code replication factor which would reflect number of nimbus 
-hosts to which the code must be replicated before starting the topology. With replication comes the issue of consistency. 
-We will treat zookeeper’s list of active topologies as our authority for topologies for which the code must exist on a 
-nimbus host. Any nimbus host that does not have all the code for all the topologies which are marked as active in zookeeper 
-will relinquish it’s lock so some other nimbus host could become leader. A background thread on all nimbus host will 
-continuously try to sync code from other hosts where the code was successfully replicated so eventually at least one nimbus 
-will accept leadership as long as at least one seed hosts exists for each active topology. 
-		
-Following steps describe code replication amongst nimbus hosts for a topology:
-* When client uploads jar, nothing changes.
-* When client submits a topology, leader nimbus calls code distributor’s upload function which will create a metafile stored 
-locally on leader nimbus. Leader nimbus will write new entries under /storm/code-distributor/topologyId to notify all 
-nonleader nimbuses that they should download this new code.
-* We wait on the leader nimbus to ensure at least N non leader nimbus has the code replicated, with a user configurable timeout.
-* When a non leader nimbus receives the notification about new code, it downloads the meta file from leader nimbus and then
-downloads the real code by calling code distributor’s download function with metafile as input.
-* Once non leader finishes downloading code, it will write an entry under /storm/code-distributor/topologyId to indicate 
-it is one of the possible places to download the code/metafile in case the leader nimbus dies.
-* leader nimbus goes ahead and does all the usual things it does as part of submit topologies.
-
-The following sequence diagram describes the communication between different components involved in code distribution.
-
-![Nimbus HA Topology Submission](images/nimbus_ha_topology_submission.png)
-
-##Thrift and Rest API 
-In order to avoid workers/supervisors/ui talking to zookeeper for getting master nimbus address we are going to modify the 
-`getClusterInfo` API so it can also return nimbus information. getClusterInfo currently returns `ClusterSummary` instance
-which has a list of `supervisorSummary` and a list of 'topologySummary` instances. We will add a list of `NimbusSummary` 
-to the `ClusterSummary`. See the structures below:
-
-```thrift
-struct ClusterSummary {
-  1: required list<SupervisorSummary> supervisors;
-  3: required list<TopologySummary> topologies;
-  4: required list<NimbusSummary> nimbuses;
-}
-
-struct NimbusSummary {
-  1: required string host;
-  2: required i32 port;
-  3: required i32 uptime_secs;
-  4: required bool isLeader;
-  5: required string version;
-}
-```
-
-This will be used by StormSubmitter, Nimbus clients,supervisors and ui to discover the current leaders and participating 
-nimbus hosts. Any nimbus host will be able to respond to these requests. The nimbus hosts can read this information once 
-from zookeeper and cache it and keep updating the cache when the watchers are fired to indicate any changes,which should 
-be rare in general case.
-
-## Configuration
-You can use nimbus ha with default configuration , however the default configuration assumes a single nimbus host so it
-trades off replication for lower topology submission latency. Depending on your use case you can adjust following configurations:
-* storm.codedistributor.class : This is a string representing fully qualified class name of a class that implements
-backtype.storm.codedistributor.ICodeDistributor. The default is set to "backtype.storm.codedistributor.LocalFileSystemCodeDistributor".
-This class leverages local file system to store both meta files and code/configs. This class adds extra load on zookeeper as even after
-downloading the code-distrbutor meta file it contacts zookeeper in order to figure out hosts from where it can download
-actual code/config and to get the current replication count. An alternative is to use 
-"org.apache.storm.hdfs.ha.codedistributor.HDFSCodeDistributor" which relies on HDFS but does not add extra load on zookeeper and will 
-make topology submission faster.
-* topology.min.replication.count : Minimum number of nimbus hosts where the code must be replicated before leader nimbus
-can mark the topology as active and create assignments. Default is 1.
-* topology.max.replication.wait.time.sec: Maximum wait time for the nimbus host replication to achieve the nimbus.min.replication.count.
-Once this time is elapsed nimbus will go ahead and perform topology activation tasks even if required nimbus.min.replication.count is not achieved. 
-The default is 60 seconds, a value of -1 indicates to wait for ever.
-*nimbus.code.sync.freq.secs: frequency at which the background thread on nimbus which syncs code for locally missing topologies will run. default is 5 minutes.
-
-Note: Even though all nimbus hosts have watchers on zookeeper to be notified immediately as soon as a new topology is available for code
-download, the callback pretty much never results in code download. In practice we have observed that the desired replication is only achieved once the background-thread runs. 
-So you should expect your topology submission time to be somewhere between 0 to (2 * nimbus.code.sync.freq.secs) for any nimbus.min.replication.count > 1.
\ No newline at end of file