[ https://issues.apache.org/jira/browse/STORM-876?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14990351#comment-14990351 ]
ASF GitHub Bot commented on STORM-876: -------------------------------------- Github user d2r commented on a diff in the pull request: https://github.com/apache/storm/pull/845#discussion_r43935420 --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj --- @@ -15,14 +15,24 @@ ;; limitations under the License. (ns backtype.storm.daemon.nimbus (:import [org.apache.thrift.server THsHaServer THsHaServer$Args]) + (:import [backtype.storm.generated KeyNotFoundException]) + (:import [backtype.storm.blobstore LocalFsBlobStore]) (:import [org.apache.thrift.protocol TBinaryProtocol TBinaryProtocol$Factory]) (:import [org.apache.thrift.exception]) (:import [org.apache.thrift.transport TNonblockingServerTransport TNonblockingServerSocket]) (:import [org.apache.commons.io FileUtils]) + (:import [javax.security.auth Subject]) + (:import [backtype.storm.security.auth NimbusPrincipal]) + (:import [java.util ArrayList]) (:import [java.nio ByteBuffer] [java.util Collections HashMap] [backtype.storm.generated NimbusSummary]) - (:import [java.io FileNotFoundException File FileOutputStream]) + (:import [java.util Iterator]) + (:import [java.nio ByteBuffer] + [java.util Collections List HashMap]) + (:import [backtype.storm.blobstore AtomicOutputStream BlobStore BlobStoreAclHandler + ClientBlobStore InputStreamWithMeta KeyFilter SyncBlobs]) --- End diff -- Blobstore, ClientBlobStore unused? > Dist Cache: Basic Functionality > ------------------------------- > > Key: STORM-876 > URL: https://issues.apache.org/jira/browse/STORM-876 > Project: Apache Storm > Issue Type: Improvement > Components: storm-core > Reporter: Robert Joseph Evans > Assignee: Robert Joseph Evans > Attachments: DISTCACHE.md, DistributedCacheDesignDocument.pdf > > > Basic functionality for the Dist Cache feature. > As part of this a new API should be added to support uploading and > downloading dist cache items. storm-core.ser, storm-conf.ser and storm.jar > should be written into the blob store instead of residing locally. We need a > default implementation of the blob store that does essentially what nimbus > currently does and does not need anything extra. But having an HDFS backend > too would be great for scalability and HA. > The supervisor should provide a way to download and manage these blobs and > provide a working directory for the worker process with symlinks to the > blobs. It should also allow the blobs to be updated and switch the symlink > atomically to point to the new blob once it is downloaded. > All of this is already done by code internal to Yahoo! we are in the process > of getting it ready to push back to open source shortly. -- This message was sent by Atlassian JIRA (v6.3.4#6332)