[ 
https://issues.apache.org/jira/browse/STORM-876?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15009421#comment-15009421
 ] 

ASF GitHub Bot commented on STORM-876:
--------------------------------------

Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45114672
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/SyncBlobs.java ---
    @@ -0,0 +1,132 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package backtype.storm.blobstore;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.utils.Utils;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.thrift.transport.TTransportException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.List;
    +import java.util.ArrayList;
    +import java.util.Map;
    +
    +/**
    + * Is called periodically and updates the nimbus with blobs based on the 
state stored inside the zookeeper
    + */
    +public class SyncBlobs {
    +  private static final Logger LOG = 
LoggerFactory.getLogger(SyncBlobs.class);
    +  private CuratorFramework zkClient;
    +  private Map conf;
    +  private BlobStore blobStore;
    +  private List<String> blobStoreKeyList = new ArrayList<String>();
    +  private List<String> zookeeperKeyList = new ArrayList<String>();
    +  private NimbusInfo nimbusInfo;
    +
    +  public SyncBlobs(BlobStore blobStore, Map conf) {
    +    this.blobStore = blobStore;
    +    this.conf = conf;
    +  }
    +
    +  public void setNimbusInfo(NimbusInfo nimbusInfo) {
    +    this.nimbusInfo = nimbusInfo;
    +  }
    +
    +  public void setZookeeperKeyList(List<String> zookeeperKeyList) {
    +    this.zookeeperKeyList = zookeeperKeyList;
    +  }
    +
    +  public void setBlobStoreKeyList(List<String> blobStoreKeyList) {
    +    this.blobStoreKeyList = blobStoreKeyList;
    +  }
    +
    +  public NimbusInfo getNimbusInfo() {
    +    return nimbusInfo;
    +  }
    +
    +  public List<String> getBlobStoreKeyList() {
    +    List<String> keyList = new ArrayList<String>();
    +    keyList.addAll(blobStoreKeyList);
    +    return keyList;
    +  }
    --- End diff --
    
    I thought that the blobstore api takes care of handling the duplicates and 
lists would be better for writing tests so that i can access them easily. Any 
way i will change to set if you like


> 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)

Reply via email to