You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@storm.apache.org by redsanket <gi...@git.apache.org> on 2015/11/03 05:38:17 UTC

[GitHub] storm pull request: [STORM-876] Blobstore API

GitHub user redsanket opened a pull request:

    https://github.com/apache/storm/pull/845

    [STORM-876] Blobstore API

    Please take a look at the blobstore API. The design docs have been uploaded earlier under the same JIRA. Currently, the blobstore API also works with Nimbus-HA. Apart from the unit tests written for blobstore and nimbus leader election, I have performed few manual tests. Having the PR early looking for suggestions and any missed failover scenarios while the work continues on other additional integration tests and better interface. The manual tests that have been performed are
    1. CRUD operations for the blobstore API. Tested for the Local and Hdfs blobstore operations to work with Nimbus-HA and whether the other nimbodes mirror the operations performed on master nimbus.
    2. Brought up 3 nimbodes and launched a word count topology. Killed the leader daemon and checked whether other non-leader takes up the leadership and continues to run.
    3. Tested whether topology waits for replication to take place before it is launched. For hdfs blobstore the blobstore.replication should be set greater than or equal to topology.min.replication to work effectively.
    4. The blobstore with nimbus-ha does not guarantee updates to sync effectively in the case of failover.
    
    Follow up code to be written.
    1. Integration test for blobstore with nimbus-ha
    2. Move the sync thread in nimbus to blobstore. Waiting on pacemaker push back as it uses some of its code and write the callback code for blobstore

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/redsanket/storm blobstore

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/storm/pull/845.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #845
    
----
commit 6e1ee30ed6e142033a8b3f264918806630da1bf1
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-02T19:59:13Z

    Squashed commit of the following:
    
    commit 3403a6da661f5ff002cf5e4571853f9b95d8e3c8
    Author: Sanket <sc...@untilservice-lm>
    Date:   Mon Nov 2 13:38:14 2015 -0600
    
        resolved unit test errors caused due to merge commits
    
    commit 898c1d64b5dcbcca2d1efc9a78e731316c1292c0
    Merge: 9909a50 8ba776b
    Author: Sanket <sc...@untilservice-lm>
    Date:   Mon Nov 2 11:41:13 2015 -0600
    
        resolved merge conflicts
    
    commit 9909a50a8b1037e16b21f45ee88c8237f33bd7b8
    Author: Sanket <sc...@untilservice-lm>
    Date:   Mon Nov 2 10:57:29 2015 -0600
    
        cleaned-up debug statements
    
    commit 3f2e1796a74799702113303e896d353ce4218c29
    Author: Sanket <sc...@untilservice-lm>
    Date:   Fri Oct 30 23:25:21 2015 -0500
    
        zookeeper state code and code clean up
    
    commit ac4db652aaad8f3ec71aca50eb31579bf90bd887
    Author: Sanket <sc...@untilservice-lm>
    Date:   Thu Oct 29 11:26:17 2015 -0500
    
        made changes to the sync blob code to fix race condition
    
    commit c96ffcaac5d247da1c77a1e1699b2d7b7f76ebf5
    Author: Sanket <sc...@untilservice-lm>
    Date:   Tue Oct 27 13:38:17 2015 -0500
    
        resolved unit test errors
    
    commit f8f7ebab8cb636a296d84b7e719c368e988063c8
    Author: Sanket <sc...@untilservice-lm>
    Date:   Mon Oct 26 22:51:11 2015 -0500
    
        added callback function
    
    commit c5f02cd54ef50039a6f4dde8f073e839b9ef7b3a
    Author: Sanket <sc...@untilservice-lm>
    Date:   Fri Oct 23 16:20:41 2015 -0500
    
        merge commit, modified unit tests to suit blob-store
    
    commit fdc77d54054470fdaa419dbedaa0d69fdf1f63ce
    Merge: eb0ea1e f75fdde
    Author: Sanket <sc...@untilservice-lm>
    Date:   Wed Oct 21 13:28:46 2015 -0500
    
        modified blobstore to accomodate nimbus ha
    
    commit eb0ea1e167ebfee94e3a0911f05c2102cdeb77b8
    Author: Sanket <sc...@untilservice-lm>
    Date:   Wed Oct 21 13:08:22 2015 -0500
    
        integrated blobstore code
    
    commit f25e7ba8139b0e2ed56ab532339d0b15b4563c6e
    Author: Sanket <sc...@untilservice-lm>
    Date:   Thu Oct 15 19:06:52 2015 -0500
    
        initial commit
    
    commit ba74039b93ee34067e2511e280a50e3958dc8741
    Author: Sanket <sc...@untilservice-lm>
    Date:   Thu Oct 8 15:04:42 2015 -0500
    
        modified thrift files to include blobstore related classes
    
    commit 73859f70e21e4acde0b3b906b53786c7d2b7a5c5
    Author: Sanket <sc...@untilservice-lm>
    Date:   Thu Oct 8 15:01:46 2015 -0500
    
        integrating blobstore and making sure it builds

commit 2506665b02d8808efd20760f1a11e99239518496
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-02T20:34:09Z

    removed log message and fixed a unit test

commit e8093acbecdfc8893bd9ac571ee8e48f3d610e80
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-03T03:32:33Z

    removed date change generated files

commit 04458b8f8d3ecb022995281630c130da30dd7b28
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-03T04:13:01Z

    indentation edit

commit a26bdf8efd94d26ba65489491594a712381373c7
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-03T04:16:06Z

    indentation edit

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45624346
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStoreAclHandler.java ---
    @@ -0,0 +1,401 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.AccessControl;
    +import backtype.storm.generated.AccessControlType;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.security.auth.AuthUtils;
    +import backtype.storm.security.auth.IPrincipalToLocal;
    +import backtype.storm.security.auth.NimbusPrincipal;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.security.Principal;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Provides common handling of acls for Blobstores.
    + * Also contains some static utility functions related to Blobstores.
    + */
    +public class BlobStoreAclHandler {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStoreAclHandler.class);
    +  private final IPrincipalToLocal _ptol;
    +
    +  public static final int READ = 0x01;
    +  public static final int WRITE = 0x02;
    +  public static final int ADMIN = 0x04;
    +  public static final List<AccessControl> WORLD_EVERYTHING =
    +      Arrays.asList(new AccessControl(AccessControlType.OTHER, READ | WRITE | ADMIN));
    +  public static final List<AccessControl> DEFAULT = new ArrayList<AccessControl>();
    +  private Set<String> _supervisors;
    +  private Set<String> _admins;
    +
    +  public BlobStoreAclHandler(Map conf) {
    +    _ptol = AuthUtils.GetPrincipalToLocalPlugin(conf);
    +    _supervisors = new HashSet<String>();
    +    _admins = new HashSet<String>();
    +    if (conf.containsKey(Config.NIMBUS_SUPERVISOR_USERS)) {
    +      _supervisors.addAll((List<String>)conf.get(Config.NIMBUS_SUPERVISOR_USERS));
    +    }
    +    if (conf.containsKey(Config.NIMBUS_ADMINS)) {
    +      _admins.addAll((List<String>)conf.get(Config.NIMBUS_ADMINS));
    +    }
    +  }
    +
    +  private static AccessControlType parseACLType(String type) {
    +    if ("other".equalsIgnoreCase(type) || "o".equalsIgnoreCase(type)) {
    +      return AccessControlType.OTHER;
    +    } else if ("user".equalsIgnoreCase(type) || "u".equalsIgnoreCase(type)) {
    +      return AccessControlType.USER;
    +    }
    +    throw new IllegalArgumentException(type+" is not a valid access control type");
    +  }
    +
    +  private static int parseAccess(String access) {
    +    int ret = 0;
    +    for (char c: access.toCharArray()) {
    +      if ('r' == c) {
    +        ret = ret | READ;
    +      } else if ('w' == c) {
    +        ret = ret | WRITE;
    +      } else if ('a' == c) {
    +        ret = ret | ADMIN;
    +      } else if ('-' == c) {
    +        //ignored
    +      } else {
    +        throw new IllegalArgumentException("");
    +      }
    +    }
    +    return ret;
    +  }
    +
    +  public static AccessControl parseAccessControl(String str) {
    +    String[] parts = str.split(":");
    +    String type = "other";
    +    String name = "";
    +    String access = "-";
    +    if (parts.length > 3) {
    +      throw new IllegalArgumentException("Don't know how to parse "+str+" into an ACL value");
    +    } else if (parts.length == 1) {
    +      type = "other";
    +      name = "";
    +      access = parts[0];
    +    } else if (parts.length == 2) {
    +      type = "user";
    +      name = parts[0];
    +      access = parts[1];
    +    } else if (parts.length == 3) {
    +      type = parts[0];
    +      name = parts[1];
    +      access = parts[2];
    +    }
    +    AccessControl ret = new AccessControl();
    +    ret.set_type(parseACLType(type));
    +    ret.set_name(name);
    +    ret.set_access(parseAccess(access));
    +    return ret;
    +  }
    +
    +  private static String accessToString(int access) {
    +    StringBuffer ret = new StringBuffer();
    +    ret.append(((access & READ) > 0) ? "r" : "-");
    +    ret.append(((access & WRITE) > 0) ? "w" : "-");
    +    ret.append(((access & ADMIN) > 0) ? "a" : "-");
    +    return ret.toString();
    +  }
    +
    +  public static String accessControlToString(AccessControl ac) {
    +    StringBuffer ret = new StringBuffer();
    +    switch(ac.get_type()) {
    +      case OTHER:
    +        ret.append("o");
    +        break;
    +      case USER:
    +        ret.append("u");
    +        break;
    +      default:
    +        throw new IllegalArgumentException("Ahh don't know what a type of "+ac.get_type()+" means ");
    +    }
    +    ret.append(":");
    +    if (ac.is_set_name()) {
    +      ret.append(ac.get_name());
    +    }
    +    ret.append(":");
    +    ret.append(accessToString(ac.get_access()));
    +    return ret.toString();
    +  }
    +
    +  public static void validateSettableACLs(String key, List<AccessControl> acls) throws AuthorizationException {
    +    Set<String> aclUsers = new HashSet<>();
    +    List<String> duplicateUsers = new ArrayList<>();
    +    for (AccessControl acl : acls) {
    +      String aclUser = acl.get_name();
    +      if (aclUser != null && !aclUser.isEmpty() && !aclUsers.add(aclUser)) {
    +        LOG.error("'{}' user can't appear more than once in the ACLs", aclUser);
    +        duplicateUsers.add(aclUser);
    +      }
    +    }
    +    if (duplicateUsers.size() > 0) {
    +      String errorMessage  = "user " + Arrays.toString(duplicateUsers.toArray())
    +          + " can't appear more than once in the ACLs for key [" + key +"].";
    +      throw new AuthorizationException(errorMessage);
    +    }
    +  }
    +
    +  private Set<String> constructUserFromPrincipals(Subject who) {
    +    Set<String> user = new HashSet<String>();
    +    if (who != null) {
    +      for (Principal p : who.getPrincipals()) {
    +        user.add(_ptol.toLocal(p));
    +      }
    +    }
    +    return user;
    +  }
    +
    +  private boolean isAdmin(Subject who) {
    +    Set<String> user = constructUserFromPrincipals(who);
    +    for (String u : user) {
    +      if (_admins.contains(u)) {
    +        return true;
    +      }
    +    }
    +    return false;
    +  }
    +
    +  private boolean isReadOperation(int operation) {
    +    if (operation == 1) {
    +      return true;
    +    }
    +    return false;
    +  }
    +
    +  private boolean isSupervisor(Subject who, int operation) {
    +    Set<String> user = constructUserFromPrincipals(who);
    +    if (isReadOperation(operation)) {
    +      for (String u : user) {
    +        if (_supervisors.contains(u)) {
    +          return true;
    +        }
    +      }
    +    }
    +    return false;
    +  }
    +
    +  private boolean isNimbus(Subject who) {
    +    Set<Principal> principals = null;
    +    boolean isNimbusInstance = false;
    +    if (who != null) {
    +      principals = who.getPrincipals();
    +      for (Principal principal : principals) {
    +        if (principal instanceof NimbusPrincipal) {
    +          isNimbusInstance = true;
    +        }
    +      }
    +    }
    +    return isNimbusInstance;
    +  }
    +
    +  public boolean checkForValidUsers(Subject who, int mask) {
    +    if (isNimbus(who) || isAdmin(who) || isSupervisor(who,mask)) {
    +      return true;
    +    }
    +    return false;
    +  }
    +
    +  /**
    +   * The user should be able to see the metadata if and only if they have any of READ, WRITE, or ADMIN
    +   */
    +  public void validateUserCanReadMeta(List<AccessControl> acl, Subject who, String key) throws AuthorizationException {
    +    hasAnyPermissions(acl, (READ|WRITE|ADMIN), who, key);
    +  }
    +
    +  /**
    +   * Validates if the user has any of the permissions
    +   * mentioned in the mask.
    +   * @param acl ACL for the key.
    +   * @param mask mask holds the cummulative value of
    +   * READ = 1, WRITE = 2 or ADMIN = 4 permissions.
    +   * mask = 1 implies READ privilege.
    +   * mask = 5 implies READ and ADMIN privileges.
    +   * @param who Is the user against whom the permissions
    +   * are validated for a key using the ACL and the mask.
    +   * @param key Key used to identify the blob.
    +   * @throws AuthorizationException
    +   */
    +  public void hasAnyPermissions(List<AccessControl> acl, int mask, Subject who, String key) throws AuthorizationException {
    +    Set<String> user = constructUserFromPrincipals(who);
    +    LOG.debug("user {}", user);
    +    if (checkForValidUsers(who, mask)) {
    +     return;
    +    }
    +    for (AccessControl ac : acl) {
    +      int allowed = getAllowed(ac, user);
    +      LOG.debug(" user: {} allowed: {} key: {}", user, allowed, key);
    +      if ((allowed & mask) > 0) {
    +        return;
    +      }
    +    }
    +    throw new AuthorizationException(
    +            user + " does not have access to " + key);
    +  }
    +
    +  /**
    +   * Validates if the user has atleast the set of permissions
    +   * mentioned in the mask.
    +   * @param acl ACL for the key.
    +   * @param mask mask holds the cummulative value of
    --- End diff --
    
    `cumulative`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r44989679
  
    --- Diff: external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStoreFile.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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 org.apache.storm.hdfs.blobstore;
    +
    +import backtype.storm.blobstore.BlobStoreFile;
    +import backtype.storm.generated.SettableBlobMeta;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FileContext;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Options;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.fs.permission.FsPermission;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.util.regex.Matcher;
    +
    +public class HdfsBlobStoreFile extends BlobStoreFile {
    +  public static final Logger LOG = LoggerFactory.getLogger(HdfsBlobStoreFile.class);
    +
    +  private final String _key;
    +  private final boolean _isTmp;
    +  private final Path _path;
    +  private Long _modTime = null;
    +  private final boolean _mustBeNew;
    +  private final Configuration _hadoopConf;
    +  private final FileSystem _fs;
    +  private SettableBlobMeta meta;
    +
    +  // files are world-wide readable and owner writable
    +  final public static FsPermission BLOBSTORE_FILE_PERMISSION =
    +      FsPermission.createImmutable((short) 0644); // rw-r--r--
    +
    +  public HdfsBlobStoreFile(Path base, String name, Configuration hconf) {
    --- End diff --
    
    Same 2-spacing here. If there are more files using 2-spacing I won't mention them.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on the pull request:

    https://github.com/apache/storm/pull/845#issuecomment-160730596
  
    +1, thanks for your work and patience, @redsanket 
    
    @revans2, @kishorvpatil , @unsleepy22 , @ptgoetz, It looks as if your comments have been addressed.
    
    @redsanket , Since there are a lot of commits on this branch, would you squash them now before we merge?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45790060
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/KeySequenceNumber.java ---
    @@ -0,0 +1,227 @@
    +/**
    + * 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.zookeeper.CreateMode;
    +import org.apache.zookeeper.ZooDefs;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.nio.ByteBuffer;
    +import java.util.TreeSet;
    +import java.util.Map;
    +import java.util.List;
    +
    +/**
    + * Class hands over the key sequence number which implies the number of updates made to a blob.
    + * The information regarding the keys and the sequence number which represents the number of updates are
    + * stored within the zookeeper in the following format.
    + * /storm/blobstore/key_name/nimbushostport-sequencenumber
    + * Example:
    + * If there are two nimbodes with nimbus.seeds:leader,non-leader are set,
    + * then the state inside the zookeeper is eventually stored as:
    + * /storm/blobstore/key1/leader:8080-1
    + * /storm/blobstore/key1/non-leader:8080-1
    + * indicates that a new blob with the name key1 has been created on the leader
    + * nimbus and the non-leader nimbus syncs after a call back is triggered by attempting
    + * to download the blob and finally updates its state inside the zookeeper.
    + *
    + * A watch is placed on the /storm/blobstore/key1 and the znodes leader:8080-1 and
    + * non-leader:8080-1 are ephemeral which implies that these nodes exist only until the
    + * connection between the corresponding nimbus and the zookeeper persist. If in case the
    + * nimbus crashes the node disappears under /storm/blobstore/key1.
    + *
    + * The sequence number for the keys are handed over based on the following scenario:
    + * Lets assume there are three nimbodes up and running, one being the leader and the other
    + * being the non-leader.
    + *
    + * 1. Create is straight forward.
    + * Check whether the znode -> /storm/blobstore/key1 has been created or not. It implies
    + * the blob has not been created yet. If not created, it creates it and updates the zookeeper
    + * states under /storm/blobstore/key1 and /storm/blobstoremaxkeysequencenumber/key1.
    + * The znodes it creates on these nodes are /storm/blobstore/key1/leader:8080-1,
    + * /storm/blobstore/key1/non-leader:8080-1 and /storm/blobstoremaxkeysequencenumber/key1/1.
    + * The later holds the global sequence number across all nimbodes more like a static variable
    + * indicating the true value of number of updates for a blob. This node helps to maintain sanity in case
    + * leadership changes due to crashing.
    + *
    + * 2. Delete does not require to hand over the sequence number.
    + *
    + * 3. Finally, the update has few scenarios.
    + *
    + *  The class implements a TreeSet. The basic idea is if all the nimbodes have the same
    + *  sequence number for the blob, then the number of elements in the set is 1 which holds
    + *  the latest value of sequence number. If the number of elements are greater than 1 then it
    + *  implies that there is sequence mismatch and there is need for syncing the blobs across
    + *  nimbodes.
    + *
    + *  The logic for handing over sequence numbers based on the state are described as follows
    + *  Here consider Nimbus-1 alias as N1 and Nimbus-2 alias as N2.
    + *  Scenario 1:
    + *  Example: Normal create/update scenario
    + *  Operation     Nimbus-1:state     Nimbus-2:state     Seq-Num-Nimbus-1  Seq-Num-Nimbus-2          Max-Seq-Num
    + *  Create-Key1   alive - Leader     alive              1                                           1
    + *  Sync          alive - Leader     alive              1                 1 (callback -> download)  1
    + *  Update-Key1   alive - Leader     alive              2                 1                         2
    + *  Sync          alive - Leader     alive              2                 2 (callback -> download)  2
    + *
    + *  Scenario 2:
    + *  Example: Leader nimbus crash followed by leader election, update and ex-leader restored again
    + *  Operation     Nimbus-1:state     Nimbus-2:state     Seq-Num-Nimbus-1  Seq-Num-Nimbus-2          Max-Seq-Num
    + *  Create        alive - Leader     alive              1                                           1
    + *  Sync          alive - Leader     alive              1                 1 (callback -> download)  1
    + *  Update        alive - Leader     alive              2                 1                         2
    + *  Sync          alive - Leader     alive              2                 2 (callback -> download)  2
    + *  Update        alive - Leader     alive              3                 2                         3
    + *  Crash         crash - Leader     alive              3                 2                         3
    + *  New - Leader  crash              alive - Leader     3 (Invalid)       2                         3
    + *  Update        crash              alive - Leader     3 (Invalid)       4 (max-seq-num + 1)       4
    + *  N1-Restored   alive              alive - Leader     0                 4                         4
    + *  Sync          alive              alive - Leader     4                 4                         4
    + *
    + *  Scenario 3:
    + *  Example: Leader nimbus crash followed by leader election, update and ex-leader restored again
    + *  Operation     Nimbus-1:state     Nimbus-2:state     Seq-Num-Nimbus-1  Seq-Num-Nimbus-2          Max-Seq-Num
    + *  Create        alive - Leader     alive              1                                           1
    + *  Sync          alive - Leader     alive              1                 1 (callback -> download)  1
    + *  Update        alive - Leader     alive              2                 1                         2
    + *  Sync          alive - Leader     alive              2                 2 (callback -> download)  2
    + *  Update        alive - Leader     alive              3                 2                         3
    + *  Crash         crash - Leader     alive              3                 2                         3
    + *  Elect Leader  crash              alive - Leader     3 (Invalid)       2                         3
    + *  N1-Restored   alive              alive - Leader     3                 2                         3
    + *  Read/Update   alive              alive - Leader     3                 4 (Downloads from N1)     4
    + *  Sync          alive              alive - Leader     4 (callback)      4                         4
    + *  Here the download is triggered whenever an operation corresponding to the blob is triggered on the
    + *  nimbus like a read or update operation. Here, in the read/update call it is hard to know which call
    + *  is read or update. Hence, by incrementing the sequence number to max-seq-num + 1 we ensure that the
    + *  synchronization happens appropriately and all nimbodes have the same blob.
    + */
    --- End diff --
    
    Yeah, +1000 on the documentation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45124346
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -1767,6 +1900,169 @@
                                       topology-id
                                       (doto (GetInfoOptions.) (.set_num_err_choice NumErrorsChoice/ALL))))
     
    +      ;;Blobstore implementation code
    +      (^String beginCreateBlob [this
    +                                ^String blob-key
    +                                ^SettableBlobMeta blob-meta]
    +        (let [session-id (uuid)]
    +          (.put (:blob-uploaders nimbus)
    +            session-id
    +            (->> (ReqContext/context)
    +              (.subject)
    +              (.createBlob (:blob-store nimbus) blob-key blob-meta)))
    +          (log-message "Created blob for " blob-key
    +            " with session id " session-id)
    +          (str session-id)))
    +
    +      (^String beginUpdateBlob [this ^String blob-key]
    +        (let [^AtomicOutputStream os (->> (ReqContext/context)
    +                                       (.subject)
    +                                       (.updateBlob (:blob-store nimbus)
    +                                         blob-key))]
    +          (let [session-id (uuid)]
    +            (.put (:blob-uploaders nimbus) session-id os)
    +            (log-message "Created upload session for " blob-key
    +              " with id " session-id)
    +            (str session-id))))
    +
    +      (^void createStateInZookeeper [this ^String blob-key]
    +        (let [storm-cluster-state (:storm-cluster-state nimbus)
    +              blob-store (:blob-store nimbus)
    +              nimbus-host-port-info (:nimbus-host-port-info nimbus)
    +              conf (:conf nimbus)]
    +          (if (instance? LocalFsBlobStore blob-store)
    +              (.setup-blobstore! storm-cluster-state blob-key nimbus-host-port-info (get-version-for-key blob-key nimbus-host-port-info conf)))
    +          (log-debug "Created state in zookeeper" storm-cluster-state blob-store nimbus-host-port-info)))
    +
    +      (^void uploadBlobChunk [this ^String session ^ByteBuffer blob-chunk]
    +        (let [uploaders (:blob-uploaders nimbus)]
    +          (if-let [^AtomicOutputStream os (.get uploaders session)]
    +            (let [chunk-array (.array blob-chunk)
    +                  remaining (.remaining blob-chunk)
    +                  array-offset (.arrayOffset blob-chunk)
    +                  position (.position blob-chunk)]
    +              (.write os chunk-array (+ array-offset position) remaining)
    +              (.put uploaders session os))
    +            (throw-runtime "Blob for session "
    +              session
    +              " does not exist (or timed out)"))))
    +
    +      (^void finishBlobUpload [this ^String session]
    +        (if-let [^AtomicOutputStream os (.get (:blob-uploaders nimbus) session)]
    +          (do
    +            (.close os)
    +            (log-message "Finished uploading blob for session "
    +              session
    +              ". Closing session.")
    +            (.remove (:blob-uploaders nimbus) session))
    +          (throw-runtime "Blob for session "
    +            session
    +            " does not exist (or timed out)")))
    +
    +      (^void cancelBlobUpload [this ^String session]
    +        (if-let [^AtomicOutputStream os (.get (:blob-uploaders nimbus) session)]
    +          (do
    +            (.cancel os)
    +            (log-message "Canceled uploading blob for session "
    +              session
    +              ". Closing session.")
    +            (.remove (:blob-uploaders nimbus) session))
    +          (throw-runtime "Blob for session "
    +            session
    +            " does not exist (or timed out)")))
    +
    +      (^ReadableBlobMeta getBlobMeta [this ^String blob-key]
    +        (let [^ReadableBlobMeta ret (->> (ReqContext/context)
    +                                      (.subject)
    +                                      (.getBlobMeta (:blob-store nimbus)
    +                                        blob-key))]
    +          ret))
    +
    +      (^void setBlobMeta [this ^String blob-key ^SettableBlobMeta blob-meta]
    +        (->> (ReqContext/context)
    +          (.subject)
    +          (.setBlobMeta (:blob-store nimbus) blob-key blob-meta)))
    +
    +      (^BeginDownloadResult beginBlobDownload [this ^String blob-key]
    +        (let [^InputStreamWithMeta is (->> (ReqContext/context)
    +                                        (.subject)
    +                                        (.getBlob (:blob-store nimbus)
    +                                          blob-key))]
    +          (let [session-id (uuid)
    +                ret (BeginDownloadResult. (.getVersion is) (str session-id))]
    +            (.set_data_size ret (.getFileLength is))
    +            (.put (:blob-downloaders nimbus) session-id (BufferInputStream. is ^Integer (Utils/getInt (conf STORM-BLOBSTORE-INPUTSTREAM-BUFFER-SIZE-BYTES) (int 65536))))
    +            (log-message "Created download session for " blob-key
    +              " with id " session-id)
    +            ret)))
    +
    +      (^ByteBuffer downloadBlobChunk [this ^String session]
    +        (let [downloaders (:blob-downloaders nimbus)
    +              ^BufferInputStream is (.get downloaders session)]
    +          (when-not is
    +            (throw (RuntimeException.
    +                     "Could not find input stream for session " session)))
    +          (let [ret (.read is)]
    +            (.put downloaders session is)
    +            (when (empty? ret)
    +              (.close is)
    +              (.remove downloaders session))
    +            (log-debug "Sending " (alength ret) " bytes")
    +            (ByteBuffer/wrap ret))))
    +
    +      (^void deleteBlob [this ^String blob-key]
    +        (let [subject (->> (ReqContext/context)
    +                           (.subject))]
    +        (.deleteBlob (:blob-store nimbus) blob-key subject)
    +        (when (instance? LocalFsBlobStore blob-store)
    +          (.remove-blobstore-key! (:storm-cluster-state nimbus) blob-key)
    +          (.remove-key-version! (:storm-cluster-state nimbus) blob-key))
    +        (log-message "Deleted blob for key " blob-key)))
    +
    +      (^ListBlobsResult listBlobs [this ^String session]
    +        (let [listers (:blob-listers nimbus)
    +              ^Iterator keys-it (if (clojure.string/blank? session)
    +                                  (->> (ReqContext/context)
    +                                    (.subject)
    +                                    (.listKeys (:blob-store nimbus)))
    +                                  (.get listers session))
    +              _ (or keys-it (throw-runtime "Blob list for session "
    +                              session
    +                              " does not exist (or timed out)"))
    +
    +              ;; Create a new session id if the user gave an empty session string.
    +              ;; This is the use case when the user wishes to list blobs
    +              ;; starting from the beginning.
    +              session (if (clojure.string/blank? session)
    +                        (let [new-session (uuid)]
    +                          (log-message "Creating new session for downloading list " new-session)
    +                          new-session)
    +                        session)]
    +          (if-not (.hasNext keys-it)
    +            (do
    +              (.remove listers session)
    +              (log-message "No more blobs to list for session " session)
    +              ;; A blank result communicates that there are no more blobs.
    +              (ListBlobsResult. (ArrayList. 0) (str session)))
    +            (let [^List list-chunk (->> keys-it
    +                                     (iterator-seq)
    +                                     (take 100) ;; Limit to next 100 keys
    +                                     (ArrayList.))
    +                  _ (log-message session " downloading " (.size list-chunk) " entries")]
    +              (.put listers session keys-it)
    +              (ListBlobsResult. list-chunk (str session))))))
    +
    +      (^int getBlobReplication [this ^String blob-key]
    +        (->> (ReqContext/context)
    +          (.subject)
    +          (.getBlobReplication (:blob-store nimbus) blob-key)))
    +
    +      (^int updateBlobReplication [this ^String blob-key ^int replication]
    +        (->> (ReqContext/context)
    +          (.subject)
    +          (.updateBlobReplication (:blob-store nimbus) blob-key replication)))
    +      ;;Blobstore implementation code ends
    --- End diff --
    
    Remove this comment?  It should be clear whether a method is for the blobstore, and if we really want to separate the code, we could move these methods to their own namespace and `extend` the nimbus instance.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45269138
  
    --- Diff: storm-core/src/jvm/backtype/storm/Config.java ---
    @@ -980,6 +980,126 @@
         public static final String SUPERVISOR_SLOTS_PORTS = "supervisor.slots.ports";
     
         /**
    +     * What blobstore implementation the supervisor should use.
    +     */
    +    @isString
    +    public static final String SUPERVISOR_BLOBSTORE = "supervisor.blobstore.class";
    +    public static final Object SUPERVISOR_BLOBSTORE_SCHEMA = String.class;
    +
    +    /**
    +     * 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.
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String SUPERVISOR_LOCALIZER_CACHE_TARGET_SIZE_MB = "supervisor.localizer.cache.target.size.mb";
    +
    +    /**
    +     * The distributed cache cleanup interval. Controls how often it scans to attempt to cleanup
    +     * anything over the cache target size.
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String SUPERVISOR_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS = "supervisor.localizer.cleanup.interval.ms";
    +
    +    /**
    +     * What blobstore implementation the storm client should use.
    +     */
    +    @isString
    +    public static final String CLIENT_BLOBSTORE = "client.blobstore.class";
    +
    +    /**
    +     * What blobstore download parallelism the supervisor should use.
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String SUPERVISOR_BLOBSTORE_DOWNLOAD_THREAD_COUNT = "supervisor.blobstore.download.thread.count";
    +
    +    /**
    +     * What blobstore download parallelism the supervisor should use.
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String SUPERVISOR_BLOBSTORE_DOWNLOAD_MAX_RETRIES = "supervisor.blobstore.download.max_retries";
    --- End diff --
    
    Check javadoc comment


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43787675
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -346,53 +394,124 @@
           [(.getNodeId slot) (.getPort slot)]
           )))
     
    +(defn- get-nimbus-subject []
    +  (let [nimbus-subject (Subject.)
    +        nimbus-principal (NimbusPrincipal.)
    +        principals (.getPrincipals nimbus-subject)]
    +    (.add principals nimbus-principal)
    +    nimbus-subject))
    +
    +(defn- get-metadata-version [blob-store key subject]
    +  (let [blob-meta (.getBlobMeta blob-store key subject)]
    +    (.get_version blob-meta)))
    +
    +(defn get-key-list-from-blob-store [blob-store]
    +  (let [key-iter (.listKeys blob-store (get-nimbus-subject))
    +        keys (iterator-seq key-iter)]
    +    (if (not-nil? keys)
    +      (java.util.ArrayList. keys)
    +      [])))
    --- End diff --
    
    Here in one place you're returning a java ArrayList, and in the other you're returning a clojure vector.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45615908
  
    --- Diff: external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java ---
    @@ -0,0 +1,383 @@
    +/**
    + * 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 org.apache.storm.hdfs.blobstore;
    +
    +import backtype.storm.Config;
    +import backtype.storm.blobstore.AtomicOutputStream;
    +import backtype.storm.blobstore.AtomicOutputStream;
    +import backtype.storm.blobstore.BlobStore;
    +import backtype.storm.blobstore.BlobStoreAclHandler;
    +import backtype.storm.blobstore.BlobStoreFile;
    +import backtype.storm.blobstore.InputStreamWithMeta;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.utils.NimbusClient;
    +import backtype.storm.utils.Utils;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.security.UserGroupInformation;
    +import org.apache.thrift7.TBase;
    --- End diff --
    
    I think the org.apache.thrift is shaded to org.apache.thrift7
    Since the hdfs blob store is moved to a package outside storm-core, it might be the only
    way to access it unless if you have any suggesstions


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by ptgoetz <gi...@git.apache.org>.
Github user ptgoetz commented on the pull request:

    https://github.com/apache/storm/pull/845#issuecomment-161736488
  
    @redsanket First of all thank you for contributing this and making it through the most epic code review we've had to date. :)
    
    I'm +1 for merging this as is, but I'd like to see two follow up tasks (can be handled as separate JIRAs):
    
    1. Merge the contents of the DISTCACHE.md and design document attached to the JIRA and add them to Storm's documentation.
    2. Add an example topology to storm-starter (e.g. one with a bolt that uses the blobstore to lookup values from a sample file, etc. that may change over time).
    
    We should probably address those 2 items before a 1.0 release.
    
    This is a big addition. I'd encourage other committers to review this as well, if for no other reason than to familiarize yourself with the functionality that's being added.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45760644
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/KeyFilter.java ---
    @@ -0,0 +1,22 @@
    +/**
    + * 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;
    +
    +public interface KeyFilter<R> {
    +  public R filter(String key);
    --- End diff --
    
    @d2r addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r44991284
  
    --- Diff: storm-core/src/clj/backtype/storm/command/blobstore.clj ---
    @@ -0,0 +1,162 @@
    +;; 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.
    +(ns backtype.storm.command.blobstore
    +  (:import [java.io InputStream OutputStream])
    +  (:use [backtype.storm config])
    +  (:import [backtype.storm.generated SettableBlobMeta AccessControl AuthorizationException
    +            KeyNotFoundException])
    +  (:import [backtype.storm.blobstore BlobStoreAclHandler])
    +  (:use [clojure.string :only [split]])
    +  (:use [clojure.tools.cli :only [cli]])
    +  (:use [clojure.java.io :only [copy input-stream output-stream]])
    +  (:use [backtype.storm blobstore log util])
    --- End diff --
    
    Since this is a new file, can we merge all the `use`s and `import`s into one `use` and one `import` form?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45267773
  
    --- Diff: storm-core/src/jvm/backtype/storm/Config.java ---
    @@ -980,6 +980,126 @@
         public static final String SUPERVISOR_SLOTS_PORTS = "supervisor.slots.ports";
     
         /**
    +     * What blobstore implementation the supervisor should use.
    +     */
    +    @isString
    +    public static final String SUPERVISOR_BLOBSTORE = "supervisor.blobstore.class";
    +    public static final Object SUPERVISOR_BLOBSTORE_SCHEMA = String.class;
    --- End diff --
    
    We don't use the `_SCHEMA` notation anymore with the new attribute implementation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46101459
  
    --- Diff: pom.xml ---
    @@ -469,7 +475,11 @@
                     <artifactId>curator-client</artifactId>
                     <version>${curator.version}</version>
                 </dependency>
    -
    +            <dependency>
    +                <groupId>org.apache.curator</groupId>
    +                <artifactId>curator-test</artifactId>
    +                <version>${curator.version}</version>
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43765777
  
    --- Diff: pom.xml ---
    @@ -646,11 +646,92 @@
                     <version>${thrift.version}</version>
                     <scope>compile</scope>
                 </dependency>
    +			<!-- used by examples/storm-starter -->
    +		    <dependency>
    +		      <groupId>junit</groupId>
    +		      <artifactId>junit</artifactId>
    +		      <version>4.11</version>
    --- End diff --
    
    There is a ${junit.version} we should be using instead of hard coding 4.11


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by kishorvpatil <gi...@git.apache.org>.
Github user kishorvpatil commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43821497
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/HdfsBlobStore.java ---
    @@ -0,0 +1,369 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.*;
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.utils.NimbusClient;
    +import backtype.storm.utils.Utils;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.security.UserGroupInformation;
    +import org.apache.thrift.TBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.ByteArrayOutputStream;
    +import java.io.FileNotFoundException;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.security.AccessController;
    +import java.security.PrivilegedAction;
    +import java.util.Iterator;
    +import java.util.Map;
    +
    +import static backtype.storm.blobstore.BlobStoreAclHandler.*;
    --- End diff --
    
    explicit import


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45425624
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/ClientBlobStore.java ---
    @@ -0,0 +1,69 @@
    +/**
    + * 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.Config;
    +import backtype.storm.daemon.Shutdownable;
    +import backtype.storm.generated.AccessControl;
    +import backtype.storm.generated.AccessControlType;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.utils.NimbusClient;
    +import backtype.storm.utils.Utils;
    --- End diff --
    
    Unused imports:
    * `Config`
    * `AccessControl`
    * `AccessControlType`
    * `Utils`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on the pull request:

    https://github.com/apache/storm/pull/845#issuecomment-159413532
  
    backtype.storm.utils.Utils
    
    After an upmerge, it looks like we cleaned up some duplicate method declarations.  However, much of the remaining code is duplicated elsewhere.
    
    
    ```
     664     public static <T> T thriftDeserialize(Class c, byte[] b) {
     665         try {
     666             T ret = (T) c.newInstance();
     667             TDeserializer des = getDes();
     668             des.deserialize((TBase) ret, b);
     669             return ret;
     670         } catch (Exception e) {
     671             throw new RuntimeException(e);
     672         }
     673
     674     }
    ```
    This should just call `Utils.thriftDeserialize(c, b, 0, b.length)`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45427729
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/KeyVersion.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * 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.zookeeper.CreateMode;
    +import org.apache.zookeeper.ZooDefs;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.TreeSet;
    +import java.util.Map;
    +import java.util.List;
    +
    +/**
    + * Class hands over the version of the key to be stored within the zookeeper
    + */
    +public class KeyVersion {
    +  private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
    +  private final String BLOBSTORE_SUBTREE="/blobstore";
    +  private final String BLOBSTORE_KEY_COUNTER_SUBTREE="/blobstorekeycounter";
    +  private String key;
    +  private NimbusInfo nimbusInfo;
    +
    +  public KeyVersion(String key, NimbusInfo nimbusInfo) {
    +    this.key = key;
    +    this.nimbusInfo = nimbusInfo;
    +  }
    +
    +  public int getKeyVersion(Map conf) {
    +    TreeSet<Integer> versions = new TreeSet<Integer>();
    +    CuratorFramework zkClient = Utils.createZKClient(conf);
    +    try {
    +      // Key has not been created yet and it is the first time it is being created
    +      if(zkClient.checkExists().forPath(BLOBSTORE_SUBTREE + "/" + key) == null) {
    +        zkClient.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT)
    +                .withACL(ZooDefs.Ids.OPEN_ACL_UNSAFE).forPath(BLOBSTORE_KEY_COUNTER_SUBTREE + "/" + key + "/" + 1);
    +        return 1;
    +      }
    +
    +      // When all nimbodes go down and one or few of them come up
    +      // Unfortunately there might not be an exact way to know which one contains the most updated blob
    +      // if all go down which is unlikely. Hence there might be a need to update the blob if all go down
    +      List<String> stateInfoList = zkClient.getChildren().forPath(BLOBSTORE_SUBTREE + "/" + key);
    +      LOG.debug("stateInfoList {} stateInfoList {}", stateInfoList.size(), stateInfoList);
    --- End diff --
    
    Check log message.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43896288
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStoreAclHandler.java ---
    @@ -0,0 +1,382 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.AccessControl;
    +import backtype.storm.generated.AccessControlType;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.security.auth.AuthUtils;
    +import backtype.storm.security.auth.IPrincipalToLocal;
    +import backtype.storm.security.auth.NimbusPrincipal;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.security.Principal;
    +import java.util.*;
    +
    +/**
    + * Provides common handling of acls for Blobstores.
    + * Also contains some static utility functions related to Blobstores.
    + */
    +public class BlobStoreAclHandler {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStoreAclHandler.class);
    +  private final IPrincipalToLocal _ptol;
    +
    +  public static final int READ = 0x01;
    +  public static final int WRITE = 0x02;
    +  public static final int ADMIN = 0x04;
    +  public static final List<AccessControl> WORLD_EVERYTHING =
    +      Arrays.asList(new AccessControl(AccessControlType.OTHER, READ | WRITE | ADMIN));
    +  public static final List<AccessControl> DEFAULT = new ArrayList<AccessControl>();
    +  private Set<String> _supervisors;
    +  private Set<String> _admins;
    +
    +  public BlobStoreAclHandler(Map conf) {
    +    _ptol = AuthUtils.GetPrincipalToLocalPlugin(conf);
    +    _supervisors = new HashSet<String>();
    +    _admins = new HashSet<String>();
    +    if (conf.containsKey(Config.NIMBUS_SUPERVISOR_USERS)) {
    +      _supervisors.addAll((List<String>)conf.get(Config.NIMBUS_SUPERVISOR_USERS));
    +    }
    +    if (conf.containsKey(Config.NIMBUS_ADMINS)) {
    +      _admins.addAll((List<String>)conf.get(Config.NIMBUS_ADMINS));
    +    }
    +  }
    +
    +  private static AccessControlType parseACLType(String type) {
    +    if ("other".equalsIgnoreCase(type) || "o".equalsIgnoreCase(type)) {
    +      return AccessControlType.OTHER;
    +    } else if ("user".equalsIgnoreCase(type) || "u".equalsIgnoreCase(type)) {
    +      return AccessControlType.USER;
    +    }
    +    throw new IllegalArgumentException(type+" is not a valid access control type");
    +  }
    +
    +  private static int parseAccess(String access) {
    +    int ret = 0;
    +    for (char c: access.toCharArray()) {
    +      if ('r' == c) {
    +        ret = ret | READ;
    +      } else if ('w' == c) {
    +        ret = ret | WRITE;
    +      } else if ('a' == c) {
    +        ret = ret | ADMIN;
    +      } else if ('-' == c) {
    +        //ignored
    +      } else {
    +        throw new IllegalArgumentException("");
    +      }
    +    }
    +    return ret;
    +  }
    +
    +  public static AccessControl parseAccessControl(String str) {
    +    String[] parts = str.split(":");
    +    String type = "other";
    +    String name = "";
    +    String access = "-";
    +    if (parts.length > 3) {
    +      throw new IllegalArgumentException("Don't know how to parse "+str+" into an ACL value");
    +    } else if (parts.length == 1) {
    +      type = "other";
    +      name = "";
    +      access = parts[0];
    +    } else if (parts.length == 2) {
    +      type = "user";
    +      name = parts[0];
    +      access = parts[1];
    +    } else if (parts.length == 3) {
    +      type = parts[0];
    +      name = parts[1];
    +      access = parts[2];
    +    }
    +    AccessControl ret = new AccessControl();
    +    ret.set_type(parseACLType(type));
    +    ret.set_name(name);
    +    ret.set_access(parseAccess(access));
    +    return ret;
    +  }
    --- End diff --
    
    This allows some shortcuts in giving ACL entries.
    `rwa` or `::rwa` -> `other::rwa`
    `alice:r` or `:alice:r` -> `user:alice:r`
    `u:bob:rwa` -> `user:bob:rwa`
    
    Looks good.  Maybe we could add it to the CLI help?  If it is already clear enough, then we do not need to.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on the pull request:

    https://github.com/apache/storm/pull/845#issuecomment-158284724
  
    Revert changes to ConfigValidationAnnotations.java; it's only white-space


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45426783
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/FileBlobStoreImpl.java ---
    @@ -0,0 +1,248 @@
    +/**
    + * 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.Config;
    +import backtype.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.FileNotFoundException;
    +import java.io.IOException;
    +import java.nio.file.Files;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Timer;
    +import java.util.TimerTask;
    +
    +/**
    + * Very basic blob store impl with no ACL handling.
    + */
    +public class FileBlobStoreImpl {
    +  private static final long FULL_CLEANUP_FREQ = 60 * 60 * 1000l;
    +  private static final int BUCKETS = 1024;
    +  private static final Logger LOG = LoggerFactory.getLogger(FileBlobStoreImpl.class);
    +  private static final Timer timer = new Timer("FileBlobStore cleanup thread", true);
    +
    +  public class KeyInHashDirIterator implements Iterator<String> {
    +    private int currentBucket = 0;
    +    private Iterator<String> it = null;
    +    private String next = null;
    +
    +    public KeyInHashDirIterator() throws IOException {
    +      primeNext();
    +    }
    +
    +    private void primeNext() throws IOException {
    +      while (it == null && currentBucket < BUCKETS) {
    +        String name = String.valueOf(currentBucket);
    +        File dir = new File(_fullPath, name);
    +        try {
    +          it = listKeys(dir);
    +        } catch (FileNotFoundException e) {
    +          it = null;
    +        }
    +        if (it == null || !it.hasNext()) {
    +          it = null;
    +          currentBucket++;
    +        } else {
    +          next = it.next();
    +        }
    +      }
    +    }
    +
    +    @Override
    +    public boolean hasNext() {
    +      return next != null;
    +    }
    +
    +    @Override
    +    public String next() {
    +      if (!hasNext()) {
    +        throw new NoSuchElementException();
    +      }
    +      String current = next;
    +      next = null;
    +      if (it != null) {
    +        if (!it.hasNext()) {
    +          it = null;
    +          currentBucket++;
    +          try {
    +            primeNext();
    +          } catch (IOException e) {
    +            throw new RuntimeException(e);
    +          }
    +        } else {
    +          next = it.next();
    +        }
    +      }
    +      return current;
    +    }
    +
    +    @Override
    +    public void remove() {
    +      throw new UnsupportedOperationException("Delete Not Supported");
    +    }
    +  }
    +
    +  private File _fullPath;
    +  private TimerTask cleanup = null;
    +
    +  public FileBlobStoreImpl(File path, Map<String, Object> conf) throws IOException {
    +    LOG.info("Creating new blob store based in {}", path);
    +    _fullPath = path;
    +    _fullPath.mkdirs();
    +    Object shouldCleanup = conf.get(Config.BLOBSTORE_CLEANUP_ENABLE);
    +    if (Utils.getBoolean(shouldCleanup, false)) {
    +      LOG.debug("Starting File blobstore cleaner");
    +      cleanup = new TimerTask() {
    +        @Override
    +        public void run() {
    +          try {
    +            fullCleanup(FULL_CLEANUP_FREQ);
    +          } catch (IOException e) {
    +            LOG.error("Error trying to cleanup", e);
    +          }
    +        }
    +      };
    +      timer.scheduleAtFixedRate(cleanup, 0, FULL_CLEANUP_FREQ);
    +    }
    +  }
    +
    +  /**
    +   * @return all keys that are available for reading.
    +   * @throws IOException on any error.
    +   */ 
    +  public Iterator<String> listKeys() throws IOException {
    +    return new KeyInHashDirIterator();
    +  }
    +
    +  /**
    +   * Get an input stream for reading a part.
    +   * @param key the key of the part to read.
    +   * @return the where to read the data from.
    +   * @throws IOException on any error
    +   */
    +  public LocalFsBlobStoreFile read(String key) throws IOException {
    +    return new LocalFsBlobStoreFile(getKeyDir(key), BlobStoreFile.BLOBSTORE_DATA_FILE);
    +  }
    +
    +  /**
    +   * Get an object tied to writing the data.
    +   * @param key the key of the part to write to.
    +   * @return an object that can be used to both write to, but also commit/cancel the operation.
    +   * @throws IOException on any error
    +   */ 
    +  public LocalFsBlobStoreFile write(String key, boolean create) throws IOException {
    +    return new LocalFsBlobStoreFile(getKeyDir(key), true, create);
    +  }
    +
    +  /**
    +   * Check if the key exists in the blob store.
    +   * @param key the key to check for
    +   * @return true if it exists else false.
    +   */
    +  public boolean exists(String key) {
    +    return getKeyDir(key).exists();
    +  }
    +  
    +  /**
    +   * Delete a key from the blob store
    +   * @param key the key to delete
    +   * @throws IOException on any error
    +   */
    +  public void deleteKey(String key) throws IOException {
    +    File keyDir = getKeyDir(key);
    +    LocalFsBlobStoreFile pf = new LocalFsBlobStoreFile(keyDir, BlobStoreFile.BLOBSTORE_DATA_FILE);
    +    pf.delete();
    +    delete(keyDir);
    +  }
    +
    +  private File getKeyDir(String key) {
    +    String hash = String.valueOf(Math.abs((long)key.hashCode()) % BUCKETS);
    +    File ret = new File(new File(_fullPath, hash), key);
    +    LOG.debug("{} Looking for {} in {}", new Object[]{_fullPath, key, hash});
    +    return ret;
    +  }
    +
    +  public void fullCleanup(long age) throws IOException {
    +    long cleanUpIfBefore = System.currentTimeMillis() - age;
    +    Iterator<String> keys = new KeyInHashDirIterator();
    +    while (keys.hasNext()) {
    +      String key = keys.next();
    +      File keyDir = getKeyDir(key);
    +      Iterator<LocalFsBlobStoreFile> i = listBlobStoreFiles(keyDir);
    +      if (!i.hasNext()) {
    +        //The dir is empty, so try to delete it, may fail, but that is OK
    +        try {
    +          keyDir.delete();
    +        } catch (Exception e) {
    +          LOG.warn("Could not delete "+keyDir+" will try again later");
    +        }
    +      }
    +      while (i.hasNext()) {
    +        LocalFsBlobStoreFile f = i.next();
    +        if (f.isTmp()) {
    +          if (f.getModTime() <= cleanUpIfBefore) {
    +            f.delete();
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  protected Iterator<LocalFsBlobStoreFile> listBlobStoreFiles(File path) throws IOException {
    +    ArrayList<LocalFsBlobStoreFile> ret = new ArrayList<LocalFsBlobStoreFile>();
    +    File[] files = path.listFiles();
    +    if (files != null) {
    --- End diff --
    
    File#listFiles does not return null.  It can return an empty array, but we handle that.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45264704
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -811,6 +993,21 @@
           :else (-> value sub-fn (clojure.string/split #"\s+")))))
     
     
    +(defn create-blobstore-links
    +  "Create symlinks in worker launch directory for all blobs"
    +  [conf storm-id port worker-id]
    +  (let [stormroot (supervisor-stormdist-root conf storm-id)
    +        storm-conf (read-supervisor-storm-conf conf storm-id)
    +        workerroot (worker-root conf worker-id)
    +        blobstore-map (storm-conf TOPOLOGY-BLOBSTORE-MAP)
    +        blob-file-names (get-blob-file-names blobstore-map)
    +        resource-file-names (cons RESOURCES-SUBDIR blob-file-names)]
    +    (log-message "Creating symlinks for worker-id: " worker-id " storm-id: "
    +      storm-id " for files(" (count resource-file-names) "): " (pr-str resource-file-names))
    +    (create-symlink! workerroot stormroot RESOURCES-SUBDIR)
    +    (doseq [file-name blob-file-names]
    +      (create-symlink! workerroot stormroot file-name file-name))))
    --- End diff --
    
    `port` parameter not used?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45760217
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStore.java ---
    @@ -0,0 +1,446 @@
    +/**
    + * 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 java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Set;
    +import java.util.regex.Pattern;
    +
    +import javax.security.auth.Subject;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import backtype.storm.daemon.Shutdownable;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +
    +/**
    + * Provides a way to store blobs that can be downloaded.
    + * Blobs must be able to be uploaded and listed from Nimbus,
    + * and downloaded from the Supervisors. It is a key value based
    + * store. Key being a string and value being the blob data.
    + *
    + * ACL checking must take place against the provided subject.
    + * If the blob store does not support Security it must validate
    + * that all ACLs set are always WORLD, everything.
    + *
    + * The users can upload their blobs through the blob store command
    + * line. The command line utilty also allows us to update,
    + * delete.
    + *
    + * Modifying the replication factor only works for HdfsBlobStore
    + * as for the LocalFsBlobStore the replication is dependent on
    + * the number of Nimbodes available.
    + */
    +public abstract class BlobStore implements Shutdownable {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStore.class);
    +  private static final Pattern KEY_PATTERN = Pattern.compile("^[\\w \\t\\.:_-]+$");
    +  protected static final String BASE_BLOBS_DIR_NAME = "blobs";
    +
    +  /**
    +   * Allows us to initialize the blob store
    +   * @param conf The storm configuration
    +   * @param baseDir The directory path to store the blobs
    +   * @param nimbusInfo Contains the nimbus host, port and leadership information.
    +   */
    +  public abstract void prepare(Map conf, String baseDir, NimbusInfo nimbusInfo);
    +
    +  /**
    +   * Creates the blob.
    +   * @param key Key for the blob.
    +   * @param meta Metadata which contains the acls information
    +   * @param who Is the subject creating the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyAlreadyExistsException
    +   */
    +  public abstract AtomicOutputStream createBlob(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException;
    +
    +  /**
    +   * Updates the blob data.
    +   * @param key Key for the blob.
    +   * @param who Is the subject having the write privilege for the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract AtomicOutputStream updateBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Gets the current version of metadata for a blob
    +   * to be viewed by the user or downloaded by the supervisor.
    +   * @param key Key for the blob.
    +   * @param who Is the subject having the read privilege for the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract ReadableBlobMeta getBlobMeta(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Sets the metadata with renewed acls for the blob.
    +   * @param key Key for the blob.
    +   * @param meta Metadata which contains the updated
    +   * acls information.
    +   * @param who Is the subject having the write privilege for the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract void setBlobMeta(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Deletes the blob data and metadata.
    +   * @param key Key for the blob.
    +   * @param who Is the subject having write privilege for the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract void deleteBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Gets the InputStream to read the blob details
    +   * @param key Key for the blob.
    +   * @param who Is the subject having the read privilege for the blob.
    +   * @return InputStreamWithMeta has the additional
    +   * file length and version information.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract InputStreamWithMeta getBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Returns an iterator with all the list of
    +   * keys currently available on the blob store.
    +   * @param who Is the subject creating the blob.
    +   * @return Iterator<String>
    +   */
    +  public abstract Iterator<String> listKeys(Subject who);
    +
    +  /**
    +   * Gets the replication factor of the blob.
    +   * @param key Key for the blob.
    +   * @param who Is the subject having the read privilege for the blob.
    +   * @return BlobReplication object containing the
    +   * replication factor for the blob.
    +   * @throws Exception
    +   */
    +  public abstract int getBlobReplication(String key, Subject who) throws Exception;
    +
    +  /**
    +   * Modifies the replication factor of the blob.
    +   * @param key Key for the blob.
    +   * @param replication The replication factor the
    +   * blob has to be set to.
    +   * @param who Is the subject having the update privilege for the blob
    +   * @return BlobReplication object containing the
    +   * updated replication factor for the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   * @throws IOException
    +   */
    +  public abstract int updateBlobReplication(String key, int replication, Subject who) throws AuthorizationException, KeyNotFoundException, IOException;
    +
    +  /**
    +   * Filters keys based on the KeyFilter
    +   * passed as the argument.
    +   * @param filter Filter passed as the
    +   * @param who Might not want to have the subject as it is not doing anything
    +   * @param <R> Type
    +   * @return Set of filtered keys
    +   */
    +  public <R> Set<R> filterAndListKeys(KeyFilter<R> filter, Subject who) {
    +    Set<R> ret = new HashSet<R>();
    +    Iterator<String> keys = listKeys(who);
    +    while (keys.hasNext()) {
    +      String key = keys.next();
    +      R filtered = filter.filter(key);
    +      if (filtered != null) {
    +        ret.add(filtered);
    +      }
    +    }
    +    return ret;
    +  }
    +
    +  /**
    +   * Validates key checking for potentially harmful patterns
    +   * @param key Key for the blob.
    +   * @throws AuthorizationException
    +   */
    +  public static final void validateKey(String key) throws AuthorizationException {
    +    if (key == null || key.isEmpty() || "..".equals(key) || ".".equals(key) || !KEY_PATTERN.matcher(key).matches()) {
    +      LOG.error("'{}' does not appear to be valid {}", key, KEY_PATTERN);
    +      throw new AuthorizationException(key+" does not appear to be a valid blob key");
    +    }
    +  }
    +
    +  /**
    +   * Wrapper called to create the blob which contains
    +   * the byte data
    +   * @param key Key for the blob.
    +   * @param data Byte data that needs to be uploaded.
    +   * @param meta Metadata which contains the acls information
    +   * @param who Is the subject creating the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyAlreadyExistsException
    +   * @throws IOException
    +   */
    +  public void createBlob(String key, byte [] data, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException, IOException {
    +    AtomicOutputStream out = null;
    +    try {
    +      out = createBlob(key, meta, who);
    +      out.write(data);
    +      out.close();
    +      out = null;
    +    } finally {
    +      if (out != null) {
    +        out.cancel();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Wrapper called to create the blob which contains
    +   * the byte data
    +   * @param key Key for the blob.
    +   * @param in InputStream from which the data is read to be
    +   * written as a part of the blob.
    +   * @param meta Metadata which contains the acls information
    +   * @param who Is the subject creating the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyAlreadyExistsException
    +   * @throws IOException
    +   */
    +  public void createBlob(String key, InputStream in, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException, IOException {
    +    AtomicOutputStream out = null;
    +    try {
    +      out = createBlob(key, meta, who);
    +      byte[] buffer = new byte[2048];
    +      int len = 0;
    +      while ((len = in.read(buffer)) > 0) {
    +        out.write(buffer, 0, len);
    +      }
    +      out.close();
    +    } catch (AuthorizationException | IOException | RuntimeException e) {
    +      out.cancel();
    --- End diff --
    
    @d2r addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r44996912
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -454,10 +456,78 @@
           (shutdown-worker supervisor id))
         ))
     
    +(defn get-blob-localname
    +  "Given the blob information either gets the localname field if it exists,
    +  else routines the default value passed in."
    +  [blob-info defaultValue]
    +  (if-let [val (if blob-info (get blob-info "localname") nil)] val defaultValue))
    +
    +(defn should-uncompress-blob?
    +  "Given the blob information returns the value of the uncompress field, handling it either being
    +  a string or a boolean value, or ifs its not specified then returns false"
    +  [blob-info]
    +  (boolean (and blob-info
    +             (if-let [val (get blob-info "uncompress")]
    +               (.booleanValue (Boolean. val))))))
    +
    +(defn remove-blob-references
    +  "Remove a reference to a blob when its no longer needed."
    +  [localizer storm-id conf]
    +  (let [storm-conf (read-supervisor-storm-conf conf storm-id)
    +        blobstore-map (storm-conf TOPOLOGY-BLOBSTORE-MAP)
    +        user (storm-conf TOPOLOGY-SUBMITTER-USER)
    +        topo-name (storm-conf TOPOLOGY-NAME)]
    +    (if blobstore-map (doseq [[k, v] blobstore-map]
    +                        (.removeBlobReference localizer
    +                          k
    +                          user
    +                          topo-name
    +                          (should-uncompress-blob? v))))))
    +
    +(defn blobstore-map-to-localresources
    +  "Returns a list of LocalResources based on the blobstore-map passed in."
    +  [blobstore-map]
    +  (if blobstore-map
    +    (for [[k, v] blobstore-map] (LocalResource. k (should-uncompress-blob? v)))
    +    ()))
    +
    +(defn add-blob-references
    +  "For each of the downloaded topologies, adds references to the blobs that the topologies are
    +  using. This is used to reconstruct the cache on restart."
    +  [localizer storm-id conf]
    +  (let [storm-conf (read-supervisor-storm-conf conf storm-id)
    +        blobstore-map (storm-conf TOPOLOGY-BLOBSTORE-MAP)
    +        user (storm-conf TOPOLOGY-SUBMITTER-USER)
    +        topo-name (storm-conf TOPOLOGY-NAME)
    +        localresources (blobstore-map-to-localresources blobstore-map)]
    +    (if blobstore-map (.addReferences localizer localresources user topo-name))))
    +
    +(defn rm-topo-files
    +  [conf storm-id localizer rm-blob-refs?]
    +  (try
    +    (if (= true rm-blob-refs?)
    +      (remove-blob-references localizer storm-id conf))
    +    (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
    +      (rmr-as-user conf storm-id (supervisor-stormdist-root conf storm-id))
    +      (rmr (supervisor-stormdist-root conf storm-id)))
    +    (catch Exception e (log-message e (str "Exception removing: " storm-id)))))
    +
    +(defn verify-downloaded-files [conf localizer assigned-storm-ids all-downloaded-storm-ids]
    +  "Method written to check for the files exists to avoid supervisor crashing
    +   Also makes sure there is no necessity for locking"
    +  (remove nil?
    +    (into #{}
    +      (for [storm-id all-downloaded-storm-ids
    +            :let [rm-blob-refs? false]
    +            :when (contains? assigned-storm-ids storm-id)]
    +        (if (not (required-topo-files-exist? conf storm-id))
    +          (do
    +            (log-debug "Files not present in topology directory")
    +            (rm-topo-files conf storm-id localizer rm-blob-refs?) storm-id))))))
    --- End diff --
    
    storm-id should get its own line.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43935460
  
    --- 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])
    +  (:import [java.io FileNotFoundException File FileOutputStream FileInputStream])
    --- End diff --
    
    FileNotFoundException unused?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45272888
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStore.java ---
    @@ -0,0 +1,446 @@
    +/**
    + * 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 java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Set;
    +import java.util.regex.Pattern;
    +
    +import javax.security.auth.Subject;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import backtype.storm.daemon.Shutdownable;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +
    +/**
    + * Provides a way to store blobs that can be downloaded.
    + * Blobs must be able to be uploaded and listed from Nimbus,
    + * and downloaded from the Supervisors. It is a key value based
    + * store. Key being a string and value being the blob data.
    + *
    + * ACL checking must take place against the provided subject.
    + * If the blob store does not support Security it must validate
    + * that all ACLs set are always WORLD, everything.
    + *
    + * The users can upload their blobs through the blob store command
    + * line utility. The command line utilty also allows us to update,
    + * delete.
    + *
    + * Modifying the replication factor only works for HdfsBlobStore
    + * as for the LocalFsBlobStore the replication is dependent on
    + * the number of Nimbodes available.
    + */
    +public abstract class BlobStore implements Shutdownable {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStore.class);
    +  private static final Pattern KEY_PATTERN = Pattern.compile("^[\\w \\t\\.:_-]+$");
    +  protected static final String BASE_BLOBS_DIR_NAME = "blobs";
    +
    +  /**
    +   * Allows us to initialize the blob store
    +   * @param conf
    +   * @param baseDir
    +   */
    +  public abstract void prepare(Map conf, String baseDir, NimbusInfo nimbusInfo);
    +
    +  /**
    +   * Creates the blob.
    +   * @param key Key for the blob.
    +   * @param meta Metadata which contains the acls information
    +   * @param who Is the subject creating the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyAlreadyExistsException
    +   */
    +  public abstract AtomicOutputStream createBlob(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException;
    +
    +  /**
    +   * Updates the blob data.
    +   * @param key Key for the blob.
    +   * @param who Is the subject creating the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract AtomicOutputStream updateBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Gets the current version of metadata for a blob
    +   * to be viewed by the user or downloaded by the supervisor.
    +   * @param key Key for the blob.
    +   * @param who Is the subject creating the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract ReadableBlobMeta getBlobMeta(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Sets the metadata with renewed acls for the blob.
    +   * @param key Key for the blob.
    +   * @param meta Metadata which contains the updated
    +   * acls information.
    +   * @param who Is the subject creating the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract void setBlobMeta(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Deletes the blob data and metadata.
    +   * @param key Key for the blob.
    +   * @param who Is the subject creating the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract void deleteBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Gets the current version of metadata for a blob
    +   * to be viewed by the user or downloaded by the supervisor.
    +   * @param key Key for the blob.
    +   * @param who Is the subject creating the blob.
    +   * @return InputStreamWithMeta has the additional
    +   * file length and version information.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract InputStreamWithMeta getBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Returns an iterator with all the list of
    +   * keys currently available on the blob store.
    +   * @param who Is the subject creating the blob.
    +   * @return Iterator<String>
    +   */
    +  public abstract Iterator<String> listKeys(Subject who);
    +
    +  /**
    +   * Gets the replication factor of the blob.
    +   * @param key Key for the blob.
    +   * @param who Is the subject creating the blob.
    +   * @return BlobReplication object containing the
    +   * replication factor for the blob.
    +   * @throws Exception
    +   */
    +  public abstract int getBlobReplication(String key, Subject who) throws Exception;
    --- End diff --
    
    Same question about `who` param.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45431937
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/KeyVersion.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * 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.zookeeper.CreateMode;
    +import org.apache.zookeeper.ZooDefs;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.TreeSet;
    +import java.util.Map;
    +import java.util.List;
    +
    +/**
    + * Class hands over the version of the key to be stored within the zookeeper
    + */
    +public class KeyVersion {
    +  private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
    +  private final String BLOBSTORE_SUBTREE="/blobstore";
    +  private final String BLOBSTORE_KEY_COUNTER_SUBTREE="/blobstorekeycounter";
    +  private String key;
    +  private NimbusInfo nimbusInfo;
    +
    +  public KeyVersion(String key, NimbusInfo nimbusInfo) {
    +    this.key = key;
    +    this.nimbusInfo = nimbusInfo;
    +  }
    +
    +  public int getKeyVersion(Map conf) {
    +    TreeSet<Integer> versions = new TreeSet<Integer>();
    +    CuratorFramework zkClient = Utils.createZKClient(conf);
    +    try {
    +      // Key has not been created yet and it is the first time it is being created
    +      if(zkClient.checkExists().forPath(BLOBSTORE_SUBTREE + "/" + key) == null) {
    +        zkClient.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT)
    +                .withACL(ZooDefs.Ids.OPEN_ACL_UNSAFE).forPath(BLOBSTORE_KEY_COUNTER_SUBTREE + "/" + key + "/" + 1);
    +        return 1;
    +      }
    +
    +      // When all nimbodes go down and one or few of them come up
    +      // Unfortunately there might not be an exact way to know which one contains the most updated blob
    +      // if all go down which is unlikely. Hence there might be a need to update the blob if all go down
    +      List<String> stateInfoList = zkClient.getChildren().forPath(BLOBSTORE_SUBTREE + "/" + key);
    +      LOG.debug("stateInfoList {} stateInfoList {}", stateInfoList.size(), stateInfoList);
    +      if(stateInfoList.isEmpty()) {
    +        return getKeyVersionCounterValue(zkClient, key);
    +      }
    +
    +      LOG.debug("stateInfoSize {}", stateInfoList.size());
    +      // In all other cases check for the latest version on the nimbus and assign the version
    +      // check if all are have same version, if not assign the highest version
    +      for (String stateInfo:stateInfoList) {
    +        versions.add(Integer.parseInt(Utils.normalizeVersionInfo(stateInfo)[1]));
    +      }
    +
    +      int currentCounter = getKeyVersionCounterValue(zkClient, key);
    +      // This condition returns version when a nimbus crashes and comes up
    +      if (!checkIfStateContainsCurrentNimbusHost(stateInfoList, nimbusInfo) && !nimbusInfo.isLeader()) {
    +        if (versions.last() < currentCounter) {
    +          return currentCounter;
    +        } else {
    +          return currentCounter - 1;
    +        }
    +      }
    +      // Condition checks for an update scenario
    +      if (stateInfoList.size() >= 1 && versions.size() == 1) {
    +        if (versions.first() < getKeyVersionCounterValue(zkClient, key)) {
    +          incrementCounter(zkClient, key, currentCounter);
    +          return currentCounter + 1;
    +        } else {
    +          incrementCounter(zkClient, key, currentCounter);
    +          return versions.first() + 1;
    --- End diff --
    
    I need help understanding what is happening here.  I think what would help me is an explanation of the relationship between the values stored under `BLOBSTORE_KEY_COUNTER_SUBTREE` and the versions derived from the data stored under `BLOBSTORE_SUBTREE`.  The comments about handling crashing nimbus daemons does not make it clear enough to me.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r44995542
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -326,16 +331,29 @@
                                              (log-error t "Error when processing event")
                                              (exit-process! 20 "Error when processing an event")
                                              ))
    +   :blob-update-timer (mk-timer :kill-fn (fn [t]
    +                                           (log-error t "Error when processing blob-update")
    +                                           (exit-process! 20 "Error when processing a blob-update")
    --- End diff --
    
    Do we want a different error code for blob-update?
    Maybe not, I'm not sure. The error message is different than the others.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45106673
  
    --- 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;
    +  }
    +
    +  public List<String> getZookeeperKeyList() {
    +    List<String> keyList = new ArrayList<String>();
    +    keyList.addAll(zookeeperKeyList);
    +    return keyList;
    +  }
    +
    +  public synchronized void syncBlobs() {
    +    try {
    +    LOG.debug("Sync blobs - blobstore {} keys {} zookeeperkeys {}", blobStore, getBlobStoreKeyList(), getZookeeperKeyList());
    +    zkClient = Utils.createZKClient(conf);
    +    deleteKeyListFromBlobStoreNotOnZookeeper(getBlobStoreKeyList(), getZookeeperKeyList());
    +    updateKeyListForBlobStore(getBlobStoreKeyList());
    +    List<String> keyListToDownload = getKeyListToDownload(getBlobStoreKeyList(), getZookeeperKeyList());
    +    LOG.debug("Key List Blobstore-> Zookeeper-> DownloadList {}-> {}-> {}", getBlobStoreKeyList(), getZookeeperKeyList(), keyListToDownload);
    +
    +    for (String key : keyListToDownload) {
    +      List<NimbusInfo> nimbusInfoList = Utils.getNimbodesWithLatestVersionOfBlob(zkClient, key);
    +      if(Utils.downloadMissingBlob(conf, blobStore, key, nimbusInfoList)) {
    +        Utils.createStateInZookeeper(conf, key, nimbusInfo);
    +      }
    +    }
    +    if (zkClient !=null) {
    +      zkClient.close();
    +    }
    +    } catch(InterruptedException exp) {
    +        LOG.error("InterruptedException {}", exp);
    +    } catch(TTransportException exp) {
    +        throw new RuntimeException(exp);
    +    } catch(Exception exp) {
    +      // Should we log or throw exception
    +        throw new RuntimeException(exp);
    --- End diff --
    
    Let's just throw RTE here, unless there is something special we can do here for this Exception?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43933729
  
    --- Diff: storm-core/src/clj/backtype/storm/command/blobstore.clj ---
    @@ -0,0 +1,163 @@
    +;; 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.
    +(ns backtype.storm.command.blobstore
    +  (:import [java.io InputStream OutputStream])
    +  (:use [backtype.storm config])
    +  (:import [backtype.storm.generated SettableBlobMeta AccessControl AuthorizationException
    +            KeyNotFoundException])
    +  (:import [backtype.storm.blobstore BlobStoreAclHandler])
    +  (:use [clojure.string :only [split]])
    +  (:use [clojure.tools.cli :only [cli]])
    +  (:use [clojure.java.io :only [copy input-stream output-stream]])
    +  (:use [backtype.storm blobstore log util])
    +  (:gen-class))
    +
    +(defn update-blob-from-stream
    +  "Update a blob in the blob store from an InputStream"
    +  [key ^InputStream in]
    +  (with-configured-blob-client blobstore
    +    (let [out (.updateBlob blobstore key)]
    +      (try 
    +        (copy in out)
    +        (.close out)
    +        (catch Exception e
    +          (log-message e)
    +          (.cancel out)
    +          (throw e))))))
    +
    +(defn create-blob-from-stream
    +  "Create a blob in the blob store from an InputStream"
    +  [key ^InputStream in ^SettableBlobMeta meta]
    +  (with-configured-blob-client blobstore
    +    (let [out (.createBlob blobstore key meta)]
    +      (try 
    +        (copy in out)
    +        (.close out)
    +        (catch Exception e
    +          (.cancel out)
    +          (throw e))))))
    +
    +(defn read-blob
    +  "Read a blob in the blob store and write to an OutputStream"
    +  [key ^OutputStream out]
    +  (with-configured-blob-client blobstore
    +    (with-open [in (.getBlob blobstore key)]
    +      (copy in out))))
    +
    +(defn as-access-control
    +  "Convert a parameter to an AccessControl object"
    +  [param]
    +  (BlobStoreAclHandler/parseAccessControl (str param)))
    +
    +(defn as-acl
    +  [param]
    +  (map as-access-control (split param #",")))
    +
    +(defn access-control-str
    +  [^AccessControl acl]
    +  (BlobStoreAclHandler/accessControlToString acl))
    +
    +(defn read-cli [args]
    +  (let [[{file :file} [key] _] (cli args ["-f" "--file" :default nil])]
    +    (if file
    +      (with-open [f (output-stream file)]
    +        (read-blob key f))
    +      (read-blob key System/out))))
    +
    +(defn update-cli [args]
    +  (let [[{file :file} [key] _] (cli args ["-f" "--file" :default nil])]
    +    (if file
    +      (with-open [f (input-stream file)]
    +        (update-blob-from-stream key f))
    +      (update-blob-from-stream key System/in))
    +    (log-message "Successfully updated " key)))
    +
    +(defn create-cli [args]
    +  (let [[{file :file acl :acl replication-factor :replication-fctr} [key] _] (cli args ["-f" "--file" :default nil]
    +                                                  ["-a" "--acl" :default [] :parse-fn as-acl]
    +                                                  ["-r" "--replication-factor" :default -1 :parse-fn parse-int])
    +        meta (doto (SettableBlobMeta. acl)
    +                   (.set_replication_factor replication-factor))]
    +    (log-message "Creating " key " with ACL " (pr-str (map access-control-str acl)))
    +    (if file
    +      (with-open [f (input-stream file)]
    +        (create-blob-from-stream key f meta))
    +      (create-blob-from-stream key System/in meta))
    +    (log-message "Successfully created " key)))
    +
    +(defn delete-cli [args]
    +  (with-configured-blob-client blobstore
    +    (doseq [key args]
    +      (.deleteBlob blobstore key)
    +      (log-message "deleted " key))))
    +
    +(defn list-cli [args]
    +  (with-configured-blob-client blobstore
    +    (let [keys (if (empty? args) (iterator-seq (.listKeys blobstore)) args)]
    +      (doseq [key keys]
    +        (try
    +          (let [meta (.getBlobMeta blobstore key)
    +                version (.get_version meta)
    +                acl (.get_acl (.get_settable meta))]
    +            (log-message key " " version " " (pr-str (map access-control-str acl))))
    +          (catch AuthorizationException ae
    +            (if-not (empty? args) (log-message "ACCESS DENIED to key: " key)))
    +          (catch KeyNotFoundException knf
    +            (if-not (empty? args) (log-message key " NOT FOUND"))))))))
    +
    +(defn set-acl-cli [args]
    +  (let [[{set-acl :set} [key] _]
    +           (cli args ["-s" "--set" :default [] :parse-fn as-acl])]
    +    (with-configured-blob-client blobstore
    +      (let [meta (.getBlobMeta blobstore key)
    +            acl (.get_acl (.get_settable meta))
    +            new-acl (if set-acl set-acl acl)
    +            new-meta (SettableBlobMeta. new-acl)]
    +        (log-message "Setting ACL for " key " to " (pr-str (map access-control-str new-acl)))
    +        (.setBlobMeta blobstore key new-meta)))))
    +
    +(defn rep-cli [args]
    +  (let [sub-command (first args)
    +        new-args (rest args)]
    +    (with-configured-blob-client blobstore
    +      (condp = sub-command
    +      "--read" (let [key (first new-args)
    +                     blob-replication (.getBlobReplication blobstore key)
    +                     replication-factor (.get_replication blob-replication)]
    +                     (log-message "Current replication factor " replication-factor)
    +                     replication-factor)
    +      "--update" (let [[{replication-factor :replication-factor} [key] _]
    +                        (cli new-args ["-r" "--replication-factor" :parse-fn parse-int])]
    +                     (if (nil? replication-factor)
    +                       (throw (RuntimeException. (str "Please set the replication factor")))
    +                       (let [blob-replication (.updateBlobReplication blobstore key replication-factor)
    +                             repl-ftr (.get_replication blob-replication)]
    +                         (log-message "Replication factor is set to " repl-ftr)
    +                         repl-ftr)))
    +      :else (throw (RuntimeException. (str sub-command " is not a supported blobstore command")))))))
    --- End diff --
    
    nit: not a supported subcommand of blobstore replication


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r44996393
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -454,10 +456,78 @@
           (shutdown-worker supervisor id))
         ))
     
    +(defn get-blob-localname
    +  "Given the blob information either gets the localname field if it exists,
    +  else routines the default value passed in."
    +  [blob-info defaultValue]
    +  (if-let [val (if blob-info (get blob-info "localname") nil)] val defaultValue))
    +
    +(defn should-uncompress-blob?
    +  "Given the blob information returns the value of the uncompress field, handling it either being
    +  a string or a boolean value, or ifs its not specified then returns false"
    +  [blob-info]
    +  (boolean (and blob-info
    +             (if-let [val (get blob-info "uncompress")]
    +               (.booleanValue (Boolean. val))))))
    +
    +(defn remove-blob-references
    +  "Remove a reference to a blob when its no longer needed."
    +  [localizer storm-id conf]
    +  (let [storm-conf (read-supervisor-storm-conf conf storm-id)
    +        blobstore-map (storm-conf TOPOLOGY-BLOBSTORE-MAP)
    +        user (storm-conf TOPOLOGY-SUBMITTER-USER)
    +        topo-name (storm-conf TOPOLOGY-NAME)]
    +    (if blobstore-map (doseq [[k, v] blobstore-map]
    +                        (.removeBlobReference localizer
    +                          k
    +                          user
    +                          topo-name
    +                          (should-uncompress-blob? v))))))
    +
    +(defn blobstore-map-to-localresources
    +  "Returns a list of LocalResources based on the blobstore-map passed in."
    +  [blobstore-map]
    +  (if blobstore-map
    +    (for [[k, v] blobstore-map] (LocalResource. k (should-uncompress-blob? v)))
    +    ()))
    --- End diff --
    
    Body should be on its own line. 
    What is the empty pair of parens here for?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43774215
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -130,22 +166,30 @@
     (defn inbox [nimbus]
       (master-inbox (:conf nimbus)))
     
    -(defn- read-storm-conf [conf storm-id]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -    (merge conf
    -       (clojurify-structure
    -         (Utils/fromCompressedJsonConf
    -           (FileUtils/readFileToByteArray
    -             (File. (master-stormconf-path stormroot))))))))
    +(defn- get-subject []
    +  (let [req (ReqContext/context)]
    +    (.subject req)))
    +
    +(defn- read-storm-conf [conf storm-id blob-store]
    +  (clojurify-structure
    +    (Utils/fromCompressedJsonConf
    +      (.readBlob blob-store (master-stormconf-key storm-id) (get-subject)))))
     
     (declare delay-event)
     (declare mk-assignments)
     
    +(defn- get-key-list-from-id [conf id]
    +  (log-debug "set keys id = " id "set = " #{(master-stormcode-key id) (master-stormjar-key id) (master-stormconf-key id)})
    +  (if (local-mode? conf)
    +    [(master-stormcode-key id) (master-stormconf-key id)]
    +    [(master-stormcode-key id) (master-stormjar-key id) (master-stormconf-key id)]
    +    ))
    --- End diff --
    
    put these on the previous line please.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45425181
  
    --- Diff: external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java ---
    @@ -0,0 +1,383 @@
    +/**
    + * 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 org.apache.storm.hdfs.blobstore;
    +
    +import backtype.storm.Config;
    +import backtype.storm.blobstore.AtomicOutputStream;
    +import backtype.storm.blobstore.AtomicOutputStream;
    +import backtype.storm.blobstore.BlobStore;
    +import backtype.storm.blobstore.BlobStoreAclHandler;
    +import backtype.storm.blobstore.BlobStoreFile;
    +import backtype.storm.blobstore.InputStreamWithMeta;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.utils.NimbusClient;
    +import backtype.storm.utils.Utils;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.security.UserGroupInformation;
    +import org.apache.thrift7.TBase;
    --- End diff --
    
    Are we still using org.apache.thrift7, or have we changed to org.apache.thrift?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on the pull request:

    https://github.com/apache/storm/pull/845#issuecomment-159406843
  
    backtype.storm.utils.Utils
    ```
     535         try {
     536             ReadableBlobMeta metadata = cb.getBlobMeta(key);
     537             nimbusBlobVersion = metadata.get_version();
     538         } catch (AuthorizationException | KeyNotFoundException exp) {
     539             throw exp;
     540         }
    ```
    
    Don't need `try ... catch` if we just throw whatever we catch.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43937803
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -130,22 +162,40 @@
     (defn inbox [nimbus]
       (master-inbox (:conf nimbus)))
     
    -(defn- read-storm-conf [conf storm-id]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -    (merge conf
    -       (clojurify-structure
    -         (Utils/fromCompressedJsonConf
    -           (FileUtils/readFileToByteArray
    -             (File. (master-stormconf-path stormroot))))))))
    +(defn- get-subject []
    +  (let [req (ReqContext/context)]
    +    (.subject req)))
    +
    +(defn- read-storm-conf [conf storm-id blob-store]
    +  (clojurify-structure
    +    (Utils/fromCompressedJsonConf
    +      (.readBlob blob-store (master-stormconf-key storm-id) (get-subject)))))
     
     (declare delay-event)
     (declare mk-assignments)
     
    +(defn nimbus-subject
    +  []
    +  (let [subject (Subject.)
    +        principal (NimbusPrincipal.)
    +        principals (.getPrincipals subject)]
    +    (.add principals principal)
    +    subject))
    +
    +(def get-nimbus-subject
    +  (nimbus-subject))
    +
    +(defn- get-key-list-from-id [conf id]
    +  (log-debug "set keys id = " id "set = " #{(master-stormcode-key id) (master-stormjar-key id) (master-stormconf-key id)})
    +  (if (local-mode? conf)
    +    [(master-stormcode-key id) (master-stormconf-key id)]
    +    [(master-stormcode-key id) (master-stormjar-key id) (master-stormconf-key id)]))
    --- End diff --
    
    minor: arg list on own line


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45759337
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/FileBlobStoreImpl.java ---
    @@ -0,0 +1,248 @@
    +/**
    + * 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.Config;
    +import backtype.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.FileNotFoundException;
    +import java.io.IOException;
    +import java.nio.file.Files;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Timer;
    +import java.util.TimerTask;
    +
    +/**
    + * Very basic blob store impl with no ACL handling.
    + */
    +public class FileBlobStoreImpl {
    +  private static final long FULL_CLEANUP_FREQ = 60 * 60 * 1000l;
    +  private static final int BUCKETS = 1024;
    +  private static final Logger LOG = LoggerFactory.getLogger(FileBlobStoreImpl.class);
    +  private static final Timer timer = new Timer("FileBlobStore cleanup thread", true);
    --- End diff --
    
    @d2r addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r44989185
  
    --- Diff: external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java ---
    @@ -0,0 +1,381 @@
    +/**
    + * 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 org.apache.storm.hdfs.blobstore;
    +
    +import backtype.storm.Config;
    +import backtype.storm.blobstore.AtomicOutputStream;
    +import backtype.storm.blobstore.AtomicOutputStream;
    +import backtype.storm.blobstore.BlobStore;
    +import backtype.storm.blobstore.BlobStoreAclHandler;
    +import backtype.storm.blobstore.BlobStoreFile;
    +import backtype.storm.blobstore.InputStreamWithMeta;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.utils.NimbusClient;
    +import backtype.storm.utils.Utils;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.security.UserGroupInformation;
    +import org.apache.thrift7.TBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.ByteArrayOutputStream;
    +import java.io.FileNotFoundException;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.security.AccessController;
    +import java.security.PrivilegedAction;
    +import java.util.Iterator;
    +import java.util.Map;
    +
    +import static backtype.storm.blobstore.BlobStoreAclHandler.ADMIN;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.READ;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.WRITE;
    +
    +/**
    + * Provides a HDFS file system backed blob store implementation.
    + * Note that this provides an api for having HDFS be the backing store for the blobstore,
    + * it is not a service/daemon.
    + */
    +public class HdfsBlobStore extends BlobStore {
    +  public static final Logger LOG = LoggerFactory.getLogger(HdfsBlobStore.class);
    +  private static final String DATA_PREFIX = "data_";
    +  private static final String META_PREFIX = "meta_";
    +  private BlobStoreAclHandler _aclHandler;
    +  private HdfsBlobStoreImpl _hbs;
    +  private Subject _localSubject;
    +  private Map conf;
    +
    +  /*
    +   * Get the subject from Hadoop so we can use it to validate the acls. There is no direct
    +   * interface from UserGroupInformation to get the subject, so do a doAs and get the context.
    +   * We could probably run everything in the doAs but for now just grab the subject.
    +   */
    --- End diff --
    
    Can we turn this into an actual javadoc-style comment?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45761185
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/KeyVersion.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * 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.zookeeper.CreateMode;
    +import org.apache.zookeeper.ZooDefs;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.TreeSet;
    +import java.util.Map;
    +import java.util.List;
    +
    +/**
    + * Class hands over the version of the key to be stored within the zookeeper
    + */
    +public class KeyVersion {
    +  private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
    +  private final String BLOBSTORE_SUBTREE="/blobstore";
    +  private final String BLOBSTORE_KEY_COUNTER_SUBTREE="/blobstorekeycounter";
    +  private String key;
    +  private NimbusInfo nimbusInfo;
    +
    +  public KeyVersion(String key, NimbusInfo nimbusInfo) {
    +    this.key = key;
    +    this.nimbusInfo = nimbusInfo;
    +  }
    +
    +  public int getKeyVersion(Map conf) {
    +    TreeSet<Integer> versions = new TreeSet<Integer>();
    +    CuratorFramework zkClient = Utils.createZKClient(conf);
    +    try {
    +      // Key has not been created yet and it is the first time it is being created
    +      if(zkClient.checkExists().forPath(BLOBSTORE_SUBTREE + "/" + key) == null) {
    +        zkClient.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT)
    +                .withACL(ZooDefs.Ids.OPEN_ACL_UNSAFE).forPath(BLOBSTORE_KEY_COUNTER_SUBTREE + "/" + key + "/" + 1);
    +        return 1;
    +      }
    +
    +      // When all nimbodes go down and one or few of them come up
    +      // Unfortunately there might not be an exact way to know which one contains the most updated blob
    +      // if all go down which is unlikely. Hence there might be a need to update the blob if all go down
    +      List<String> stateInfoList = zkClient.getChildren().forPath(BLOBSTORE_SUBTREE + "/" + key);
    +      LOG.debug("stateInfoList {} stateInfoList {}", stateInfoList.size(), stateInfoList);
    +      if(stateInfoList.isEmpty()) {
    +        return getKeyVersionCounterValue(zkClient, key);
    +      }
    +
    +      LOG.debug("stateInfoSize {}", stateInfoList.size());
    +      // In all other cases check for the latest version on the nimbus and assign the version
    +      // check if all are have same version, if not assign the highest version
    +      for (String stateInfo:stateInfoList) {
    +        versions.add(Integer.parseInt(Utils.normalizeVersionInfo(stateInfo)[1]));
    +      }
    +
    +      int currentCounter = getKeyVersionCounterValue(zkClient, key);
    +      // This condition returns version when a nimbus crashes and comes up
    +      if (!checkIfStateContainsCurrentNimbusHost(stateInfoList, nimbusInfo) && !nimbusInfo.isLeader()) {
    +        if (versions.last() < currentCounter) {
    +          return currentCounter;
    +        } else {
    +          return currentCounter - 1;
    +        }
    +      }
    +      // Condition checks for an update scenario
    +      if (stateInfoList.size() >= 1 && versions.size() == 1) {
    +        if (versions.first() < getKeyVersionCounterValue(zkClient, key)) {
    +          incrementCounter(zkClient, key, currentCounter);
    +          return currentCounter + 1;
    +        } else {
    +          incrementCounter(zkClient, key, currentCounter);
    +          return versions.first() + 1;
    --- End diff --
    
    @d2r addressed all issues for all the above comments except
    
    Utils.java:
    
    1252     // Normalize state
    1253     public static String[] normalizeVersionInfo(String nimbusKeyVersionInfo) {
    1254       String[] normalizeVersionInfo = new String[2];
    1255       int lastIndex = nimbusKeyVersionInfo.lastIndexOf("-");
    1256       normalizeVersionInfo[0] = nimbusKeyVersionInfo.substring(0, lastIndex);
    1257       normalizeVersionInfo[1] = nimbusKeyVersionInfo.substring(lastIndex + 1);
    1258       return normalizeVersionInfo;
    1259     }
    We should make a class out of BlobVersion, with name and version as members. It's easier to understand than a String[] value. Will do in next request


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45262960
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -732,25 +863,89 @@
      (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
       (worker-launcher-and-wait conf (storm-conf TOPOLOGY-SUBMITTER-USER) ["code-dir" dir] :log-prefix (str "setup conf for " dir))))
     
    +(defn setup-blob-permission [conf storm-conf path]
    +  (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
    +    (worker-launcher-and-wait conf (storm-conf TOPOLOGY-SUBMITTER-USER) ["blob" path] :log-prefix (str "setup blob permissions for " path))))
    +
    +(defn setup-storm-code-dir [conf storm-conf dir]
    +  (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
    +    (worker-launcher-and-wait conf (storm-conf TOPOLOGY-SUBMITTER-USER) ["code-dir" dir] :log-prefix (str "setup conf for " dir))))
    --- End diff --
    
    * Param list should be on its own line
    * This is the second place we define `setup-storm-code-dir`.  Probably a merge error.  They are the same, except this second copy has proper 2-space indentation instead of 1-space.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45389705
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStoreAclHandler.java ---
    @@ -0,0 +1,387 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.AccessControl;
    +import backtype.storm.generated.AccessControlType;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.security.auth.AuthUtils;
    +import backtype.storm.security.auth.IPrincipalToLocal;
    +import backtype.storm.security.auth.NimbusPrincipal;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.security.Principal;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Provides common handling of acls for Blobstores.
    + * Also contains some static utility functions related to Blobstores.
    + */
    +public class BlobStoreAclHandler {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStoreAclHandler.class);
    +  private final IPrincipalToLocal _ptol;
    +
    +  public static final int READ = 0x01;
    +  public static final int WRITE = 0x02;
    +  public static final int ADMIN = 0x04;
    +  public static final List<AccessControl> WORLD_EVERYTHING =
    +      Arrays.asList(new AccessControl(AccessControlType.OTHER, READ | WRITE | ADMIN));
    +  public static final List<AccessControl> DEFAULT = new ArrayList<AccessControl>();
    +  private Set<String> _supervisors;
    +  private Set<String> _admins;
    +
    +  public BlobStoreAclHandler(Map conf) {
    +    _ptol = AuthUtils.GetPrincipalToLocalPlugin(conf);
    +    _supervisors = new HashSet<String>();
    +    _admins = new HashSet<String>();
    +    if (conf.containsKey(Config.NIMBUS_SUPERVISOR_USERS)) {
    +      _supervisors.addAll((List<String>)conf.get(Config.NIMBUS_SUPERVISOR_USERS));
    +    }
    +    if (conf.containsKey(Config.NIMBUS_ADMINS)) {
    +      _admins.addAll((List<String>)conf.get(Config.NIMBUS_ADMINS));
    +    }
    +  }
    +
    +  private static AccessControlType parseACLType(String type) {
    +    if ("other".equalsIgnoreCase(type) || "o".equalsIgnoreCase(type)) {
    +      return AccessControlType.OTHER;
    +    } else if ("user".equalsIgnoreCase(type) || "u".equalsIgnoreCase(type)) {
    +      return AccessControlType.USER;
    +    }
    +    throw new IllegalArgumentException(type+" is not a valid access control type");
    +  }
    +
    +  private static int parseAccess(String access) {
    +    int ret = 0;
    +    for (char c: access.toCharArray()) {
    +      if ('r' == c) {
    +        ret = ret | READ;
    +      } else if ('w' == c) {
    +        ret = ret | WRITE;
    +      } else if ('a' == c) {
    +        ret = ret | ADMIN;
    +      } else if ('-' == c) {
    +        //ignored
    +      } else {
    +        throw new IllegalArgumentException("");
    +      }
    +    }
    +    return ret;
    +  }
    +
    +  public static AccessControl parseAccessControl(String str) {
    +    String[] parts = str.split(":");
    +    String type = "other";
    +    String name = "";
    +    String access = "-";
    +    if (parts.length > 3) {
    +      throw new IllegalArgumentException("Don't know how to parse "+str+" into an ACL value");
    +    } else if (parts.length == 1) {
    +      type = "other";
    +      name = "";
    +      access = parts[0];
    +    } else if (parts.length == 2) {
    +      type = "user";
    +      name = parts[0];
    +      access = parts[1];
    +    } else if (parts.length == 3) {
    +      type = parts[0];
    +      name = parts[1];
    +      access = parts[2];
    +    }
    +    AccessControl ret = new AccessControl();
    +    ret.set_type(parseACLType(type));
    +    ret.set_name(name);
    +    ret.set_access(parseAccess(access));
    +    return ret;
    +  }
    +
    +  private static String accessToString(int access) {
    +    StringBuffer ret = new StringBuffer();
    +    ret.append(((access & READ) > 0) ? "r" : "-");
    +    ret.append(((access & WRITE) > 0) ? "w" : "-");
    +    ret.append(((access & ADMIN) > 0) ? "a" : "-");
    +    return ret.toString();
    +  }
    +
    +  public static String accessControlToString(AccessControl ac) {
    +    StringBuffer ret = new StringBuffer();
    +    switch(ac.get_type()) {
    +      case OTHER:
    +        ret.append("o");
    +        break;
    +      case USER:
    +        ret.append("u");
    +        break;
    +      default:
    +        throw new IllegalArgumentException("Ahh don't know what a type of "+ac.get_type()+" means ");
    +    }
    +    ret.append(":");
    +    if (ac.is_set_name()) {
    +      ret.append(ac.get_name());
    +    }
    +    ret.append(":");
    +    ret.append(accessToString(ac.get_access()));
    +    return ret.toString();
    +  }
    +
    +  public static void validateSettableACLs(String key, List<AccessControl> acls) throws AuthorizationException {
    +    Set<String> aclUsers = new HashSet<>();
    +    List<String> duplicateUsers = new ArrayList<>();
    +    for (AccessControl acl : acls) {
    +      String aclUser = acl.get_name();
    +      if (aclUser != null && !aclUser.isEmpty() && !aclUsers.add(aclUser)) {
    +        LOG.error("'{}' user can't appear more than once in the ACLs", aclUser);
    +        duplicateUsers.add(aclUser);
    +      }
    +    }
    +    if (duplicateUsers.size() > 0) {
    +      String errorMessage  = "user " + Arrays.toString(duplicateUsers.toArray())
    +          + " can't appear more than once in the ACLs for key [" + key +"].";
    +      throw new AuthorizationException(errorMessage);
    +    }
    +  }
    +
    +  private Set<String> constructUserFromPrincipals(Subject who) {
    +    Set<String> user = new HashSet<String>();
    +    if (who == null) {
    +      LOG.debug("in validate acl who is null");
    +    } else {
    +      LOG.debug("in validate acl: " + who);
    +    }
    +    if (who != null) {
    +      for (Principal p : who.getPrincipals()) {
    +        user.add(_ptol.toLocal(p));
    +      }
    +    }
    +    return user;
    +  }
    +
    +  private boolean isSupervisorOrAdmin(Set<String> user, int mask) {
    +    boolean isSupervisor = false;
    +    boolean isAdmin = false;
    +    for(String u : user) {
    +      if (_supervisors.contains(u)) {
    +        isSupervisor = true;
    +        break;
    +      }
    +      if (_admins.contains(u)) {
    +        isAdmin = true;
    --- End diff --
    
    Could we just return `true` here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45397478
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStore.java ---
    @@ -0,0 +1,306 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.utils.Utils;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.thrift.TBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.ByteArrayOutputStream;
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.FileNotFoundException;
    +import java.io.InputStream;
    +
    +import java.util.*;
    +
    +import static backtype.storm.blobstore.BlobStoreAclHandler.ADMIN;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.READ;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.WRITE;
    +
    +/**
    + * Provides a local file system backed blob store implementation for Nimbus.
    + */
    +public class LocalFsBlobStore extends BlobStore {
    +  public static final Logger LOG = LoggerFactory.getLogger(LocalFsBlobStore.class);
    +  private static final String DATA_PREFIX = "data_";
    +  private static final String META_PREFIX = "meta_";
    +  protected BlobStoreAclHandler _aclHandler;
    +  private final String BLOBSTORE_SUBTREE = "/blobstore/";
    +  private NimbusInfo nimbusInfo;
    +  private FileBlobStoreImpl fbs;
    +  private Map conf;
    +
    +  @Override
    +  public void prepare(Map conf, String overrideBase, NimbusInfo nimbusInfo) {
    +    this.conf = conf;
    +    this.nimbusInfo = nimbusInfo;
    +     if (overrideBase == null) {
    +      overrideBase = (String)conf.get(Config.BLOBSTORE_DIR);
    +      if (overrideBase == null) {
    +        overrideBase = (String) conf.get(Config.STORM_LOCAL_DIR);
    +      }
    +    }
    +    File baseDir = new File(overrideBase, BASE_BLOBS_DIR_NAME);
    +    try {
    +      fbs = new FileBlobStoreImpl(baseDir, conf);
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +    _aclHandler = new BlobStoreAclHandler(conf);
    +  }
    +
    +  @Override
    +  public AtomicOutputStream createBlob(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException {
    +    LOG.debug("Creating Blob for key {}", key);
    +    validateKey(key);
    +    _aclHandler.normalizeSettableBlobMeta(key, meta, who, READ | WRITE | ADMIN);
    +    BlobStoreAclHandler.validateSettableACLs(key, meta.get_acl());
    +    _aclHandler.validateACL(meta.get_acl(), READ | WRITE | ADMIN, who, key);
    +    if (fbs.exists(DATA_PREFIX+key)) {
    +      throw new KeyAlreadyExistsException(key);
    +    }
    +    BlobStoreFileOutputStream mOut = null;
    +    try {
    +      mOut = new BlobStoreFileOutputStream(fbs.write(META_PREFIX+key, true));
    +      mOut.write(Utils.thriftSerialize((TBase) meta));
    +      mOut.close();
    +      mOut = null;
    +      return new BlobStoreFileOutputStream(fbs.write(DATA_PREFIX+key, true));
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    } finally {
    +      if (mOut != null) {
    +        try {
    +          mOut.cancel();
    +        } catch (IOException e) {
    +          //Ignored
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public AtomicOutputStream updateBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    checkForBlobOrDownload(key);
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.validateACL(meta.get_acl(), WRITE, who, key);
    +    validateKey(key);
    +    try {
    +      return new BlobStoreFileOutputStream(fbs.write(DATA_PREFIX+key, false));
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  private SettableBlobMeta getStoredBlobMeta(String key) throws KeyNotFoundException {
    +    InputStream in = null;
    +    try {
    +      LocalFsBlobStoreFile pf = fbs.read(META_PREFIX+key);
    +      try {
    +        in = pf.getInputStream();
    +      } catch (FileNotFoundException fnf) {
    +        throw new KeyNotFoundException(key);
    +      }
    +      ByteArrayOutputStream out = new ByteArrayOutputStream();
    +      byte [] buffer = new byte[2048];
    +      int len;
    +      while ((len = in.read(buffer)) > 0) {
    +        out.write(buffer, 0, len);
    +      }
    +      in.close();
    +      in = null;
    +      SettableBlobMeta sbm = Utils.thriftDeserialize(SettableBlobMeta.class, out.toByteArray());
    +      return sbm;
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    } finally {
    +      if (in != null) {
    +        try {
    +          in.close();
    +        } catch (IOException e) {
    +          //Ignored
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public ReadableBlobMeta getBlobMeta(String key, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    if(!checkForBlobOrDownload(key)) {
    +      checkForBlobUpdate(key);
    +    }
    +    validateKey(key);
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.validateUserCanReadMeta(meta.get_acl(), who, key);
    +    ReadableBlobMeta rbm = new ReadableBlobMeta();
    +    rbm.set_settable(meta);
    +    try {
    +      LocalFsBlobStoreFile pf = fbs.read(DATA_PREFIX+key);
    +      rbm.set_version(pf.getModTime());
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +    return rbm;
    +  }
    +
    +  @Override
    +  public void setBlobMeta(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    checkForBlobOrDownload(key);
    +    validateKey(key);
    +    _aclHandler.normalizeSettableBlobMeta(key, meta, who, ADMIN);
    +    BlobStoreAclHandler.validateSettableACLs(key, meta.get_acl());
    +    SettableBlobMeta orig = getStoredBlobMeta(key);
    +    _aclHandler.validateACL(orig.get_acl(), ADMIN, who, key);
    +    BlobStoreFileOutputStream mOut = null;
    +    try {
    +      mOut = new BlobStoreFileOutputStream(fbs.write(META_PREFIX+key, false));
    +      mOut.write(Utils.thriftSerialize((TBase) meta));
    +      mOut.close();
    +      mOut = null;
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    } finally {
    +      if (mOut != null) {
    +        try {
    +          mOut.cancel();
    +        } catch (IOException e) {
    +          //Ignored
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void deleteBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    checkForBlobOrDownload(key);
    +    validateKey(key);
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.validateACL(meta.get_acl(), WRITE, who, key);
    +    try {
    +      fbs.deleteKey(DATA_PREFIX+key);
    +      fbs.deleteKey(META_PREFIX+key);
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override
    +  public InputStreamWithMeta getBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    if(!checkForBlobOrDownload(key)) {
    +      checkForBlobUpdate(key);
    +    }
    +    validateKey(key);
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.validateACL(meta.get_acl(), READ, who, key);
    +    try {
    +      return new BlobStoreFileInputStream(fbs.read(DATA_PREFIX+key));
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override
    +  public Iterator<String> listKeys(Subject who) {
    +    try {
    +      return new KeyTranslationIterator(fbs.listKeys(), DATA_PREFIX);
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override
    +  public void shutdown() {
    +  }
    +
    +  @Override
    +  public int getBlobReplication(String key, Subject who) throws Exception {
    +    validateKey(key);
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.validateACL(meta.get_acl(), READ, who, key);
    +    CuratorFramework zkClient = Utils.createZKClient(conf);
    +    if (zkClient.checkExists().forPath(BLOBSTORE_SUBTREE + key) == null) {
    +       zkClient.close();
    +       return 0;
    +    }
    +    int replicationCount = zkClient.getChildren().forPath(BLOBSTORE_SUBTREE + key).size();
    +    zkClient.close();
    +    return replicationCount;
    +  }
    +
    +  @Override
    +  public int updateBlobReplication(String key, int replication, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    int replicationCount = 0;
    +    validateKey(key);
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.validateACL(meta.get_acl(), READ, who, key);
    --- End diff --
    
    This should be `WRITE`, not `READ`.  The supervisor should not be allowed to update the blob replication.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43833973
  
    --- Diff: storm-core/src/clj/backtype/storm/config.clj ---
    @@ -112,6 +124,10 @@
         (FileUtils/forceMkdir (File. ret))
         ret ))
     
    +(defn read-supervisor-storm-conf-given-path
    +  [conf stormconf-path]
    +  (merge conf (clojurify-structure (Utils/fromCompressedJsonConf (FileUtils/readFileToByteArray (File. stormconf-path))))))
    --- End diff --
    
    Used it
    (defn read-supervisor-storm-conf
      [conf storm-id]
      (let [stormroot (supervisor-stormdist-root conf storm-id)
            conf-path (supervisor-stormconf-path stormroot)
            topology-path (supervisor-stormcode-path stormroot)]
        (read-supervisor-storm-conf-given-path conf conf-path)))


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43940318
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -346,53 +400,104 @@
           [(.getNodeId slot) (.getPort slot)]
           )))
     
    +(defn- get-metadata-version [blob-store key subject]
    +  (let [blob-meta (.getBlobMeta blob-store key subject)]
    +    (.get_version blob-meta)))
    +
    +(defn get-key-set-from-blob-store [blob-store]
    +  (let [key-iter (.listKeys blob-store get-nimbus-subject)]
    +    (set (iterator-seq key-iter))))
    +
     (defn- setup-storm-code [nimbus conf storm-id tmp-jar-location storm-conf topology]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -   (log-message "nimbus file location:" stormroot)
    -   (FileUtils/forceMkdir (File. stormroot))
    -   (FileUtils/cleanDirectory (File. stormroot))
    -   (setup-jar conf tmp-jar-location stormroot)
    -   (FileUtils/writeByteArrayToFile (File. (master-stormcode-path stormroot)) (Utils/serialize topology))
    -   (FileUtils/writeByteArrayToFile (File. (master-stormconf-path stormroot)) (Utils/toCompressedJsonConf storm-conf))
    -   (if (:code-distributor nimbus) (.upload (:code-distributor nimbus) stormroot storm-id))
    -   ))
    +  (let [subject (get-subject)
    +        storm-cluster-state (:storm-cluster-state nimbus)
    +        blob-store (:blob-store nimbus)
    +        jar-key (master-stormjar-key storm-id)
    +        code-key (master-stormcode-key storm-id)
    +        conf-key (master-stormconf-key storm-id)
    +        nimbus-host-port-info (:nimbus-host-port-info nimbus)]
    +    (if tmp-jar-location ;;in local mode there is no jar
    +      (do
    +        (.createBlob blob-store jar-key (FileInputStream. tmp-jar-location) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +        (if (instance? LocalFsBlobStore blob-store)
    +          (.setup-blobstore! storm-cluster-state jar-key nimbus-host-port-info (get-metadata-version blob-store jar-key subject)))))
    +    (.createBlob blob-store conf-key (Utils/toCompressedJsonConf storm-conf) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state conf-key nimbus-host-port-info (get-metadata-version blob-store conf-key subject)))
    +    (.createBlob blob-store code-key (Utils/serialize topology) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state code-key nimbus-host-port-info (get-metadata-version blob-store code-key subject)))))
    +
    +(defn- read-storm-topology [storm-id blob-store]
    +  (Utils/deserialize
    +    (.readBlob blob-store (master-stormcode-key storm-id) (get-subject)) StormTopology))
    +
    +(defn- get-blob-replication-count [blob-key nimbus]
    +  (if (:blob-store nimbus)
    +          (-> (:blob-store nimbus)
    +            (.getBlobReplication  blob-key get-nimbus-subject)
    +            (.get_replication))))
     
     (defn- wait-for-desired-code-replication [nimbus conf storm-id]
       (let [min-replication-count (conf TOPOLOGY-MIN-REPLICATION-COUNT)
             max-replication-wait-time (conf TOPOLOGY-MAX-REPLICATION-WAIT-TIME-SEC)
    -        total-wait-time (atom 0)
    -        current-replication-count (atom (if (:code-distributor nimbus) (.getReplicationCount (:code-distributor nimbus) storm-id) 0))]
    -  (if (:code-distributor nimbus)
    -    (while (and (> min-replication-count @current-replication-count)
    -             (or (= -1 max-replication-wait-time)
    -               (< @total-wait-time max-replication-wait-time)))
    +        current-replication-count-jar (if (not (local-mode? conf)) (atom
    +                                                                     (get-blob-replication-count (master-stormjar-key storm-id) nimbus))
    +                                                                     (atom min-replication-count))
    +        current-replication-count-code (atom (get-blob-replication-count (master-stormcode-key storm-id) nimbus))
    +        current-replication-count-conf (atom (get-blob-replication-count (master-stormconf-key storm-id) nimbus))
    +        total-wait-time (atom 0)]
    +    (log-message "wait for desired replication" "count"
    +      min-replication-count "wait-time" max-replication-wait-time
    +      "code" @current-replication-count-code
    +      "conf" @current-replication-count-conf
    +      "jar" @current-replication-count-jar
    +      "replication count" (get-blob-replication-count (master-stormconf-key storm-id) nimbus))
    +    (if (:blob-store nimbus)
    +      (while (and (> min-replication-count @current-replication-count-jar)
    +               (> min-replication-count @current-replication-count-code)
    +               (> min-replication-count @current-replication-count-conf)
    +               (or (= -1 max-replication-wait-time)
    +                 (< @total-wait-time max-replication-wait-time)))
    --- End diff --
    
    * Let's test for `(neg? max-replication-wait-time)`
    * Shouldn't there be an or around the checks for jar, code, conf replication counts?  We want to wait until all of them have the right number of replications, not until any of them does:
    ```Clojure
    (while
      (and 
        (or (> min-replication-count @current-replication-count-jar)
            (> min-replication-count @current-replication-count-code)
            (> min-replication-count @current-replication-count-conf))
        (or (neg? max-replication-wait-time)
            (< @total-wait-time max-replication-wait-time))))
    ```



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45759172
  
    --- Diff: storm-core/src/jvm/backtype/storm/Config.java ---
    @@ -980,6 +980,126 @@
         public static final String SUPERVISOR_SLOTS_PORTS = "supervisor.slots.ports";
     
         /**
    +     * What blobstore implementation the supervisor should use.
    +     */
    +    @isString
    +    public static final String SUPERVISOR_BLOBSTORE = "supervisor.blobstore.class";
    +    public static final Object SUPERVISOR_BLOBSTORE_SCHEMA = String.class;
    +
    +    /**
    +     * 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.
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String SUPERVISOR_LOCALIZER_CACHE_TARGET_SIZE_MB = "supervisor.localizer.cache.target.size.mb";
    +
    +    /**
    +     * The distributed cache cleanup interval. Controls how often it scans to attempt to cleanup
    +     * anything over the cache target size.
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String SUPERVISOR_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS = "supervisor.localizer.cleanup.interval.ms";
    +
    +    /**
    +     * What blobstore implementation the storm client should use.
    +     */
    +    @isString
    +    public static final String CLIENT_BLOBSTORE = "client.blobstore.class";
    +
    +    /**
    +     * What blobstore download parallelism the supervisor should use.
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String SUPERVISOR_BLOBSTORE_DOWNLOAD_THREAD_COUNT = "supervisor.blobstore.download.thread.count";
    +
    +    /**
    +     * What blobstore download parallelism the supervisor should use.
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String SUPERVISOR_BLOBSTORE_DOWNLOAD_MAX_RETRIES = "supervisor.blobstore.download.max_retries";
    +
    +    /**
    +     * The blobstore super user has all read/write/admin permissions to all blobs - user running
    +     * the blobstore.
    +     */
    +    @isString
    +    public static final String BLOBSTORE_SUPERUSER = "blobstore.superuser";
    +
    +    /**
    +     * What directory to use for the blobstore. The directory is expected to be an
    +     * absolute path when using HDFS blobstore, for LocalFsBlobStore it could be either
    +     * absolute or relative.
    +     */
    +    @isString
    +    public static final String BLOBSTORE_DIR = "blobstore.dir";
    +
    +    /**
    +     * What buffer size to use for the blobstore uploads.
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String STORM_BLOBSTORE_INPUTSTREAM_BUFFER_SIZE_BYTES = "storm.blobstore.inputstream.buffer.size.bytes";
    +
    +    /**
    +     * Enable the blobstore cleaner. Certain blobstores may only want to run the cleaner
    +     * on one daemon. Currently Nimbus handles setting this.
    +     */
    +    @isBoolean
    +    public static final String BLOBSTORE_CLEANUP_ENABLE = "blobstore.cleanup.enable";
    +
    +    /**
    +     * principal for nimbus/supervisor to use to access secure hdfs for the blobstore.
    +     */
    +    @isString
    +    public static final String BLOBSTORE_HDFS_PRINCIPAL = "blobstore.hdfs.principal";
    +
    +    /**
    +     * keytab for nimbus/supervisor to use to access secure hdfs for the blobstore.
    +     */
    +    @isString
    +    public static final String BLOBSTORE_HDFS_KEYTAB = "blobstore.hdfs.keytab";
    +
    +    /**
    +     *  Set replication factor for a blob in HDFS Blobstore Implementation
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String STORM_BLOBSTORE_REPLICATION_FACTOR = "storm.blobstore.replication.factor";
    --- End diff --
    
    @d2r 
    There is also a TOPOLOGY_BLOBSTORE_MIN_REPLICATION_FACTOR which specifies the amount of replication a blob has to undergo before the topology is launched, but on the other hand for HDFS blobstore we have to set the replication factor for the blob on a HDFS level. I could go ahead with HDFS_BLOBSTORE* but that would represent HDFS config rather than a STORM config. Welcome for suggestions


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45269983
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStore.java ---
    @@ -0,0 +1,446 @@
    +/**
    + * 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 java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Set;
    +import java.util.regex.Pattern;
    +
    +import javax.security.auth.Subject;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import backtype.storm.daemon.Shutdownable;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +
    +/**
    + * Provides a way to store blobs that can be downloaded.
    + * Blobs must be able to be uploaded and listed from Nimbus,
    + * and downloaded from the Supervisors. It is a key value based
    + * store. Key being a string and value being the blob data.
    + *
    + * ACL checking must take place against the provided subject.
    + * If the blob store does not support Security it must validate
    + * that all ACLs set are always WORLD, everything.
    + *
    + * The users can upload their blobs through the blob store command
    + * line utility. The command line utilty also allows us to update,
    --- End diff --
    
    `utility`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45624059
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStoreAclHandler.java ---
    @@ -0,0 +1,387 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.AccessControl;
    +import backtype.storm.generated.AccessControlType;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.security.auth.AuthUtils;
    +import backtype.storm.security.auth.IPrincipalToLocal;
    +import backtype.storm.security.auth.NimbusPrincipal;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.security.Principal;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Provides common handling of acls for Blobstores.
    + * Also contains some static utility functions related to Blobstores.
    + */
    +public class BlobStoreAclHandler {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStoreAclHandler.class);
    +  private final IPrincipalToLocal _ptol;
    +
    +  public static final int READ = 0x01;
    +  public static final int WRITE = 0x02;
    +  public static final int ADMIN = 0x04;
    +  public static final List<AccessControl> WORLD_EVERYTHING =
    +      Arrays.asList(new AccessControl(AccessControlType.OTHER, READ | WRITE | ADMIN));
    +  public static final List<AccessControl> DEFAULT = new ArrayList<AccessControl>();
    +  private Set<String> _supervisors;
    +  private Set<String> _admins;
    +
    +  public BlobStoreAclHandler(Map conf) {
    +    _ptol = AuthUtils.GetPrincipalToLocalPlugin(conf);
    +    _supervisors = new HashSet<String>();
    +    _admins = new HashSet<String>();
    +    if (conf.containsKey(Config.NIMBUS_SUPERVISOR_USERS)) {
    +      _supervisors.addAll((List<String>)conf.get(Config.NIMBUS_SUPERVISOR_USERS));
    +    }
    +    if (conf.containsKey(Config.NIMBUS_ADMINS)) {
    +      _admins.addAll((List<String>)conf.get(Config.NIMBUS_ADMINS));
    +    }
    +  }
    +
    +  private static AccessControlType parseACLType(String type) {
    +    if ("other".equalsIgnoreCase(type) || "o".equalsIgnoreCase(type)) {
    +      return AccessControlType.OTHER;
    +    } else if ("user".equalsIgnoreCase(type) || "u".equalsIgnoreCase(type)) {
    +      return AccessControlType.USER;
    +    }
    +    throw new IllegalArgumentException(type+" is not a valid access control type");
    +  }
    +
    +  private static int parseAccess(String access) {
    +    int ret = 0;
    +    for (char c: access.toCharArray()) {
    +      if ('r' == c) {
    +        ret = ret | READ;
    +      } else if ('w' == c) {
    +        ret = ret | WRITE;
    +      } else if ('a' == c) {
    +        ret = ret | ADMIN;
    +      } else if ('-' == c) {
    +        //ignored
    +      } else {
    +        throw new IllegalArgumentException("");
    +      }
    +    }
    +    return ret;
    +  }
    +
    +  public static AccessControl parseAccessControl(String str) {
    +    String[] parts = str.split(":");
    +    String type = "other";
    +    String name = "";
    +    String access = "-";
    +    if (parts.length > 3) {
    +      throw new IllegalArgumentException("Don't know how to parse "+str+" into an ACL value");
    +    } else if (parts.length == 1) {
    +      type = "other";
    +      name = "";
    +      access = parts[0];
    +    } else if (parts.length == 2) {
    +      type = "user";
    +      name = parts[0];
    +      access = parts[1];
    +    } else if (parts.length == 3) {
    +      type = parts[0];
    +      name = parts[1];
    +      access = parts[2];
    +    }
    +    AccessControl ret = new AccessControl();
    +    ret.set_type(parseACLType(type));
    +    ret.set_name(name);
    +    ret.set_access(parseAccess(access));
    +    return ret;
    +  }
    +
    +  private static String accessToString(int access) {
    +    StringBuffer ret = new StringBuffer();
    +    ret.append(((access & READ) > 0) ? "r" : "-");
    +    ret.append(((access & WRITE) > 0) ? "w" : "-");
    +    ret.append(((access & ADMIN) > 0) ? "a" : "-");
    +    return ret.toString();
    +  }
    +
    +  public static String accessControlToString(AccessControl ac) {
    +    StringBuffer ret = new StringBuffer();
    +    switch(ac.get_type()) {
    +      case OTHER:
    +        ret.append("o");
    +        break;
    +      case USER:
    +        ret.append("u");
    +        break;
    +      default:
    +        throw new IllegalArgumentException("Ahh don't know what a type of "+ac.get_type()+" means ");
    +    }
    +    ret.append(":");
    +    if (ac.is_set_name()) {
    +      ret.append(ac.get_name());
    +    }
    +    ret.append(":");
    +    ret.append(accessToString(ac.get_access()));
    +    return ret.toString();
    +  }
    +
    +  public static void validateSettableACLs(String key, List<AccessControl> acls) throws AuthorizationException {
    +    Set<String> aclUsers = new HashSet<>();
    +    List<String> duplicateUsers = new ArrayList<>();
    +    for (AccessControl acl : acls) {
    +      String aclUser = acl.get_name();
    +      if (aclUser != null && !aclUser.isEmpty() && !aclUsers.add(aclUser)) {
    +        LOG.error("'{}' user can't appear more than once in the ACLs", aclUser);
    +        duplicateUsers.add(aclUser);
    +      }
    +    }
    +    if (duplicateUsers.size() > 0) {
    +      String errorMessage  = "user " + Arrays.toString(duplicateUsers.toArray())
    +          + " can't appear more than once in the ACLs for key [" + key +"].";
    +      throw new AuthorizationException(errorMessage);
    +    }
    +  }
    +
    +  private Set<String> constructUserFromPrincipals(Subject who) {
    +    Set<String> user = new HashSet<String>();
    +    if (who == null) {
    +      LOG.debug("in validate acl who is null");
    +    } else {
    +      LOG.debug("in validate acl: " + who);
    +    }
    +    if (who != null) {
    +      for (Principal p : who.getPrincipals()) {
    +        user.add(_ptol.toLocal(p));
    +      }
    +    }
    +    return user;
    +  }
    +
    +  private boolean isSupervisorOrAdmin(Set<String> user, int mask) {
    +    boolean isSupervisor = false;
    +    boolean isAdmin = false;
    +    for(String u : user) {
    +      if (_supervisors.contains(u)) {
    +        isSupervisor = true;
    +        break;
    +      }
    +      if (_admins.contains(u)) {
    +        isAdmin = true;
    +        break;
    +      }
    +    }
    +    if (mask > 0 && !isAdmin) {
    +      isSupervisor = (isSupervisor && (mask == 1));
    +    }
    +    return isSupervisor || isAdmin;
    +  }
    +
    +  private boolean isNimbus(Subject who) {
    +    Set<Principal> principals = null;
    +    boolean isNimbusInstance = false;
    +    if(who != null) {
    +      principals = who.getPrincipals();
    +      for (Principal principal : principals) {
    +        if (principal instanceof NimbusPrincipal) {
    +          isNimbusInstance = true;
    +        }
    +      }
    +    }
    +    return isNimbusInstance;
    +  }
    +
    +  /**
    +   * The user should be able to see the metadata if and only if they have any of READ, WRITE, or ADMIN
    +   */
    +  public void validateUserCanReadMeta(List<AccessControl> acl, Subject who, String key)
    +      throws AuthorizationException {
    +    Set<String> user = constructUserFromPrincipals(who);
    +    if (isNimbus(who)) {
    +      return;
    +    }
    +    if (isSupervisorOrAdmin(user, -1)) {
    +      return;
    +    }
    +    for (AccessControl ac : acl) {
    +      int allowed = getAllowed(ac, user);
    +      LOG.debug(" user: {} allowed: {} key: {}", user, allowed, key);
    +      if ((allowed & (READ | WRITE | ADMIN)) > 0) {
    +        return;
    +      }
    +    }
    +    throw new AuthorizationException(
    +            user + " does not have access to " + key);
    +  }
    +
    +  public void validateAnyACL(List<AccessControl> acl, int validPermissions, Subject who, String key) throws AuthorizationException {
    +    Set<String> user = constructUserFromPrincipals(who);
    +    LOG.debug("user {}", user);
    +    if (isNimbus(who)) {
    +      return;
    +    }
    +    if(isSupervisorOrAdmin(user, validPermissions)) {
    +      return;
    +    }
    +    for (AccessControl ac : acl) {
    +      int allowed = getAllowed(ac, user);
    +      LOG.debug(" user: {} allowed: {} key: {}", user, allowed, key);
    +      if ((allowed & validPermissions) > 0) {
    +        return;
    +      }
    +    }
    +    throw new AuthorizationException(
    +            user + " does not have access to " + key);
    +  }
    +
    +  //Here all acls must match
    +  public void validateACL(List<AccessControl> acl, int mask, Subject who, String key)
    +      throws AuthorizationException {
    +    Set<String> user = constructUserFromPrincipals(who);
    +    LOG.debug("user {}", user);
    +    if (isNimbus(who)) {
    +      return;
    +    }
    +    if(isSupervisorOrAdmin(user, mask)) {
    +      return;
    +    }
    +    for (AccessControl ac : acl) {
    +      int allowed = getAllowed(ac, user);
    +      mask = ~allowed & mask;
    +      LOG.debug(" user: {} allowed: {} disallowed: {} key: {}", user, allowed, mask, key);
    +    }
    +    if (mask == 0) {
    +      return;
    +    }
    +    throw new AuthorizationException(
    +            user + " does not have " + namedPerms(mask) + " access to " + key);
    +  }
    --- End diff --
    
    Looks good.  I'll comment on javadocs separately.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45561357
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStoreAclHandler.java ---
    @@ -0,0 +1,387 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.AccessControl;
    +import backtype.storm.generated.AccessControlType;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.security.auth.AuthUtils;
    +import backtype.storm.security.auth.IPrincipalToLocal;
    +import backtype.storm.security.auth.NimbusPrincipal;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.security.Principal;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Provides common handling of acls for Blobstores.
    + * Also contains some static utility functions related to Blobstores.
    + */
    +public class BlobStoreAclHandler {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStoreAclHandler.class);
    +  private final IPrincipalToLocal _ptol;
    +
    +  public static final int READ = 0x01;
    +  public static final int WRITE = 0x02;
    +  public static final int ADMIN = 0x04;
    +  public static final List<AccessControl> WORLD_EVERYTHING =
    +      Arrays.asList(new AccessControl(AccessControlType.OTHER, READ | WRITE | ADMIN));
    +  public static final List<AccessControl> DEFAULT = new ArrayList<AccessControl>();
    +  private Set<String> _supervisors;
    +  private Set<String> _admins;
    +
    +  public BlobStoreAclHandler(Map conf) {
    +    _ptol = AuthUtils.GetPrincipalToLocalPlugin(conf);
    +    _supervisors = new HashSet<String>();
    +    _admins = new HashSet<String>();
    +    if (conf.containsKey(Config.NIMBUS_SUPERVISOR_USERS)) {
    +      _supervisors.addAll((List<String>)conf.get(Config.NIMBUS_SUPERVISOR_USERS));
    +    }
    +    if (conf.containsKey(Config.NIMBUS_ADMINS)) {
    +      _admins.addAll((List<String>)conf.get(Config.NIMBUS_ADMINS));
    +    }
    +  }
    +
    +  private static AccessControlType parseACLType(String type) {
    +    if ("other".equalsIgnoreCase(type) || "o".equalsIgnoreCase(type)) {
    +      return AccessControlType.OTHER;
    +    } else if ("user".equalsIgnoreCase(type) || "u".equalsIgnoreCase(type)) {
    +      return AccessControlType.USER;
    +    }
    +    throw new IllegalArgumentException(type+" is not a valid access control type");
    +  }
    +
    +  private static int parseAccess(String access) {
    +    int ret = 0;
    +    for (char c: access.toCharArray()) {
    +      if ('r' == c) {
    +        ret = ret | READ;
    +      } else if ('w' == c) {
    +        ret = ret | WRITE;
    +      } else if ('a' == c) {
    +        ret = ret | ADMIN;
    +      } else if ('-' == c) {
    +        //ignored
    +      } else {
    +        throw new IllegalArgumentException("");
    +      }
    +    }
    +    return ret;
    +  }
    +
    +  public static AccessControl parseAccessControl(String str) {
    +    String[] parts = str.split(":");
    +    String type = "other";
    +    String name = "";
    +    String access = "-";
    +    if (parts.length > 3) {
    +      throw new IllegalArgumentException("Don't know how to parse "+str+" into an ACL value");
    +    } else if (parts.length == 1) {
    +      type = "other";
    +      name = "";
    +      access = parts[0];
    +    } else if (parts.length == 2) {
    +      type = "user";
    +      name = parts[0];
    +      access = parts[1];
    +    } else if (parts.length == 3) {
    +      type = parts[0];
    +      name = parts[1];
    +      access = parts[2];
    +    }
    +    AccessControl ret = new AccessControl();
    +    ret.set_type(parseACLType(type));
    +    ret.set_name(name);
    +    ret.set_access(parseAccess(access));
    +    return ret;
    +  }
    +
    +  private static String accessToString(int access) {
    +    StringBuffer ret = new StringBuffer();
    +    ret.append(((access & READ) > 0) ? "r" : "-");
    +    ret.append(((access & WRITE) > 0) ? "w" : "-");
    +    ret.append(((access & ADMIN) > 0) ? "a" : "-");
    +    return ret.toString();
    +  }
    +
    +  public static String accessControlToString(AccessControl ac) {
    +    StringBuffer ret = new StringBuffer();
    +    switch(ac.get_type()) {
    +      case OTHER:
    +        ret.append("o");
    +        break;
    +      case USER:
    +        ret.append("u");
    +        break;
    +      default:
    +        throw new IllegalArgumentException("Ahh don't know what a type of "+ac.get_type()+" means ");
    +    }
    +    ret.append(":");
    +    if (ac.is_set_name()) {
    +      ret.append(ac.get_name());
    +    }
    +    ret.append(":");
    +    ret.append(accessToString(ac.get_access()));
    +    return ret.toString();
    +  }
    +
    +  public static void validateSettableACLs(String key, List<AccessControl> acls) throws AuthorizationException {
    +    Set<String> aclUsers = new HashSet<>();
    +    List<String> duplicateUsers = new ArrayList<>();
    +    for (AccessControl acl : acls) {
    +      String aclUser = acl.get_name();
    +      if (aclUser != null && !aclUser.isEmpty() && !aclUsers.add(aclUser)) {
    +        LOG.error("'{}' user can't appear more than once in the ACLs", aclUser);
    +        duplicateUsers.add(aclUser);
    +      }
    +    }
    +    if (duplicateUsers.size() > 0) {
    +      String errorMessage  = "user " + Arrays.toString(duplicateUsers.toArray())
    +          + " can't appear more than once in the ACLs for key [" + key +"].";
    +      throw new AuthorizationException(errorMessage);
    +    }
    +  }
    +
    +  private Set<String> constructUserFromPrincipals(Subject who) {
    +    Set<String> user = new HashSet<String>();
    +    if (who == null) {
    +      LOG.debug("in validate acl who is null");
    +    } else {
    +      LOG.debug("in validate acl: " + who);
    +    }
    +    if (who != null) {
    +      for (Principal p : who.getPrincipals()) {
    +        user.add(_ptol.toLocal(p));
    +      }
    +    }
    +    return user;
    +  }
    +
    +  private boolean isSupervisorOrAdmin(Set<String> user, int mask) {
    +    boolean isSupervisor = false;
    +    boolean isAdmin = false;
    +    for(String u : user) {
    +      if (_supervisors.contains(u)) {
    +        isSupervisor = true;
    +        break;
    +      }
    +      if (_admins.contains(u)) {
    +        isAdmin = true;
    +        break;
    +      }
    +    }
    +    if (mask > 0 && !isAdmin) {
    +      isSupervisor = (isSupervisor && (mask == 1));
    +    }
    +    return isSupervisor || isAdmin;
    +  }
    +
    +  private boolean isNimbus(Subject who) {
    +    Set<Principal> principals = null;
    +    boolean isNimbusInstance = false;
    +    if(who != null) {
    +      principals = who.getPrincipals();
    +      for (Principal principal : principals) {
    +        if (principal instanceof NimbusPrincipal) {
    +          isNimbusInstance = true;
    +        }
    +      }
    +    }
    +    return isNimbusInstance;
    +  }
    +
    +  /**
    +   * The user should be able to see the metadata if and only if they have any of READ, WRITE, or ADMIN
    +   */
    +  public void validateUserCanReadMeta(List<AccessControl> acl, Subject who, String key)
    +      throws AuthorizationException {
    +    Set<String> user = constructUserFromPrincipals(who);
    +    if (isNimbus(who)) {
    +      return;
    +    }
    +    if (isSupervisorOrAdmin(user, -1)) {
    +      return;
    +    }
    +    for (AccessControl ac : acl) {
    +      int allowed = getAllowed(ac, user);
    +      LOG.debug(" user: {} allowed: {} key: {}", user, allowed, key);
    +      if ((allowed & (READ | WRITE | ADMIN)) > 0) {
    +        return;
    +      }
    +    }
    +    throw new AuthorizationException(
    +            user + " does not have access to " + key);
    +  }
    --- End diff --
    
    @d2r addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by unsleepy22 <gi...@git.apache.org>.
Github user unsleepy22 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46022937
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStoreUtils.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.security.auth.NimbusPrincipal;
    +import backtype.storm.utils.NimbusClient;
    +import backtype.storm.utils.Utils;
    +import backtype.storm.utils.ZookeeperAuthInfo;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.thrift.transport.TTransportException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class BlobStoreUtils {
    +    private static final String BLOBSTORE_SUBTREE="/blobstore";
    +    private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
    +
    +    public static CuratorFramework createZKClient(Map conf) {
    +        List<String> zkServers = (List<String>) conf.get(Config.STORM_ZOOKEEPER_SERVERS);
    +        Object port = conf.get(Config.STORM_ZOOKEEPER_PORT);
    +        ZookeeperAuthInfo zkAuthInfo = new ZookeeperAuthInfo(conf);
    +        CuratorFramework zkClient = Utils.newCurator(conf, zkServers, port, (String) conf.get(Config.STORM_ZOOKEEPER_ROOT), zkAuthInfo);
    +        zkClient.start();
    +        return zkClient;
    +    }
    +
    +    public static Subject getNimbusSubject() {
    +        Subject subject = new Subject();
    +        subject.getPrincipals().add(new NimbusPrincipal());
    +        return subject;
    +    }
    +
    +    // Normalize state
    +    public static BlobKeySequenceInfo normalizeNimbusHostPortSequenceNumberInfo(String nimbusSeqNumberInfo) {
    +        BlobKeySequenceInfo keySequenceInfo = new BlobKeySequenceInfo();
    +        int lastIndex = nimbusSeqNumberInfo.lastIndexOf("-");
    +        keySequenceInfo.setNimbusHostPort(nimbusSeqNumberInfo.substring(0, lastIndex));
    +        keySequenceInfo.setSequenceNumber(nimbusSeqNumberInfo.substring(lastIndex + 1));
    +        return keySequenceInfo;
    +    }
    +
    +    // Check for latest sequence number of a key inside zookeeper and return nimbodes containing the latest sequence number
    +    public static Set<NimbusInfo> getNimbodesWithLatestSequenceNumberOfBlob(CuratorFramework zkClient, String key) throws Exception {
    +        List<String> stateInfoList = zkClient.getChildren().forPath("/blobstore/" + key);
    +        Set<NimbusInfo> nimbusInfoSet = new HashSet<NimbusInfo>();
    +        int latestSeqNumber = getLatestSequenceNumber(stateInfoList);
    +        LOG.debug("getNimbodesWithLatestSequenceNumberOfBlob stateInfo {} version {}", stateInfoList, latestSeqNumber);
    +        // Get the nimbodes with the latest version
    +        for(String state : stateInfoList) {
    +            BlobKeySequenceInfo sequenceInfo = normalizeNimbusHostPortSequenceNumberInfo(state);
    +            if (latestSeqNumber == Integer.parseInt(sequenceInfo.getSequenceNumber())) {
    +                nimbusInfoSet.add(NimbusInfo.parse(sequenceInfo.getNimbusHostPort()));
    +            }
    +        }
    +        LOG.debug("nimbusInfoList {}", nimbusInfoSet);
    +        return nimbusInfoSet;
    +    }
    +
    +    // Get sequence number details from latest sequence number of the blob
    +    public static int getLatestSequenceNumber(List<String> stateInfoList) {
    +        int seqNumber = 0;
    +        // Get latest sequence number of the blob present in the zookeeper --> possible to refactor this piece of code
    +        for (String state : stateInfoList) {
    +            BlobKeySequenceInfo sequenceInfo = normalizeNimbusHostPortSequenceNumberInfo(state);
    +            if (seqNumber < Integer.parseInt(sequenceInfo.getSequenceNumber())) {
    +                seqNumber = Integer.parseInt(sequenceInfo.getSequenceNumber());
    +                LOG.debug("Sequence Info {}", seqNumber);
    +            }
    +        }
    +        LOG.debug("Latest Sequence Number {}", seqNumber);
    +        return seqNumber;
    +    }
    +
    +    // Download missing blobs from potential nimbodes
    +    public static boolean downloadMissingBlob(Map conf, BlobStore blobStore, String key, Set<NimbusInfo> nimbusInfos)
    +            throws TTransportException {
    +        NimbusClient client = null;
    +        ReadableBlobMeta rbm = null;
    +        ClientBlobStore remoteBlobStore = null;
    +        InputStreamWithMeta in = null;
    +        Boolean isSuccess = false;
    +        LOG.debug("Download blob NimbusInfos {}", nimbusInfos);
    +        for (NimbusInfo nimbusInfo : nimbusInfos) {
    +            if(isSuccess) {
    +                break;
    +            }
    +            try {
    +                client = new NimbusClient(conf, nimbusInfo.getHost(), nimbusInfo.getPort(), null);
    +                rbm = client.getClient().getBlobMeta(key);
    +                remoteBlobStore = new NimbusBlobStore();
    +                remoteBlobStore.setClient(conf, client);
    +                in = remoteBlobStore.getBlob(key);
    +                blobStore.createBlob(key, in, rbm.get_settable(), getNimbusSubject());
    +                // if key already exists while creating the blob else update it
    +                Iterator<String> keyIterator = blobStore.listKeys();
    +                while (keyIterator.hasNext()) {
    +                    if (keyIterator.next().equals(key)) {
    +                        LOG.debug("Success creating key, {}", key);
    +                        isSuccess = true;
    +                        break;
    +                    }
    +                }
    +            } catch (IOException | AuthorizationException exception) {
    +                throw new RuntimeException(exception);
    +            } catch (KeyAlreadyExistsException kae) {
    +                LOG.info("KeyAlreadyExistsException Key: {} {}", key, kae);
    +            } catch (KeyNotFoundException knf) {
    +                // Catching and logging KeyNotFoundException because, if
    +                // there is a subsequent update and delete, the non-leader
    +                // nimbodes might throw an exception.
    +                LOG.info("KeyNotFoundException Key: {} {}", key, knf);
    +            } catch (Exception exp) {
    +                // Logging an exception while client is connecting
    +                LOG.error("Exception {}", exp);
    +            }
    +        }
    +
    +        if (!isSuccess) {
    +            LOG.error("Could not download blob with key" + key);
    +            return false;
    +        }
    +        return isSuccess;
    +    }
    +
    +    // Download updated blobs from potential nimbodes
    +    public static boolean downloadUpdatedBlob(Map conf, BlobStore blobStore, String key, Set<NimbusInfo> nimbusInfos)
    +            throws TTransportException {
    +        NimbusClient client = null;
    +        ClientBlobStore remoteBlobStore = null;
    +        InputStreamWithMeta in = null;
    +        AtomicOutputStream out = null;
    +        Boolean isSuccess = false;
    +        LOG.debug("Download blob NimbusInfos {}", nimbusInfos);
    +        for (NimbusInfo nimbusInfo : nimbusInfos) {
    +            if (isSuccess) {
    +                break;
    +            }
    +            try {
    +                client = new NimbusClient(conf, nimbusInfo.getHost(), nimbusInfo.getPort(), null);
    +                remoteBlobStore = new NimbusBlobStore();
    +                remoteBlobStore.setClient(conf, client);
    +                in = remoteBlobStore.getBlob(key);
    +                out = blobStore.updateBlob(key, getNimbusSubject());
    +                byte[] buffer = new byte[2048];
    +                int len = 0;
    +                while ((len = in.read(buffer)) > 0) {
    +                    out.write(buffer, 0, len);
    +                }
    +                if (out != null) {
    +                    out.close();
    +                }
    +                isSuccess = true;
    +            } catch (IOException | AuthorizationException exception) {
    +                throw new RuntimeException(exception);
    +            } catch (KeyNotFoundException knf) {
    +                // Catching and logging KeyNotFoundException because, if
    +                // there is a subsequent update and delete, the non-leader
    +                // nimbodes might throw an exception.
    +                LOG.info("KeyNotFoundException {}", knf);
    +            } catch (Exception exp) {
    +                // Logging an exception while client is connecting
    +                LOG.error("Exception {}", exp);
    +            }
    +        }
    +
    +        if (!isSuccess) {
    +            LOG.error("Could not update the blob with key" + key);
    +            return false;
    --- End diff --
    
    remove this line


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46100043
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/NimbusBlobStore.java ---
    @@ -0,0 +1,408 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.BeginDownloadResult;
    +import backtype.storm.generated.ListBlobsResult;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.utils.NimbusClient;
    +import backtype.storm.utils.Utils;
    +import org.apache.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.nio.ByteBuffer;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +
    +public class NimbusBlobStore extends ClientBlobStore {
    +  private static final Logger LOG = LoggerFactory.getLogger(NimbusBlobStore.class);
    +
    +  public class NimbusKeyIterator implements Iterator<String> {
    +    private ListBlobsResult listBlobs = null;
    +    private int offset = 0;
    +    private boolean eof = false;
    +    
    +    public NimbusKeyIterator(ListBlobsResult listBlobs) {
    +      this.listBlobs = listBlobs;
    +      this.eof = (listBlobs.get_keys_size() == 0);
    +    }
    +    
    +    private boolean isCacheEmpty() {
    +      return listBlobs.get_keys_size() <= offset;
    +    }
    +
    +    private void readMore() {
    +      if (!eof) {
    +        try {
    +          offset = 0;
    +          synchronized(client) {
    +            listBlobs = client.getClient().listBlobs(listBlobs.get_session());
    +          }
    +          if (listBlobs.get_keys_size() == 0) {
    +            eof = true;
    +          }
    +        } catch (TException e) {
    +          throw new RuntimeException(e);
    +        }
    +      }
    +    }
    +    
    +    @Override
    +    public synchronized boolean hasNext() {
    +      if (isCacheEmpty()) {
    +        readMore();
    +      }
    +      return !eof;
    +    }
    +
    +    @Override
    +    public synchronized String next() {
    +      if (!hasNext()) {
    +        throw new NoSuchElementException();
    +      }
    +      String ret = listBlobs.get_keys().get(offset);
    +      offset++;
    +      return ret;
    +    }
    +    
    +    @Override
    +    public void remove() {
    +      throw new UnsupportedOperationException("Delete Not Supported");
    +    }
    +  }
    +
    +  public class NimbusDownloadInputStream extends InputStreamWithMeta {
    +    private BeginDownloadResult beginBlobDownload;
    +    private byte[] buffer = null;
    +    private int offset = 0;
    +    private int end = 0;
    +    private boolean eof = false;
    +
    +    public NimbusDownloadInputStream(BeginDownloadResult beginBlobDownload) {
    +      this.beginBlobDownload = beginBlobDownload;
    +    }
    +
    +    @Override
    +    public long getVersion() throws IOException {
    +      return beginBlobDownload.get_version();
    +    }
    +
    +    @Override
    +    public synchronized int read() throws IOException {
    +      if (isEmpty()) {
    +        readMore();
    +        if (eof) {
    +          return -1;
    +        }
    +      }
    +      int length = Math.min(1, available());
    +      if (length == 0) {
    +        return -1;
    +      }
    +      int ret = buffer[offset];
    +      offset += length;
    +      return ret;
    +    }
    +    
    +    @Override 
    +    public synchronized int read(byte[] b, int off, int len) throws IOException {
    +      if (isEmpty()) {
    +        readMore();
    +        if (eof) {
    +          return -1;
    +        }
    +      }
    +      int length = Math.min(len, available());
    +      System.arraycopy(buffer, offset, b, off, length);
    +      offset += length;
    +      return length;
    +    }
    +    
    +    private boolean isEmpty() {
    +      return buffer == null || offset >= end;
    +    }
    +    
    +    private void readMore() {
    +      if (!eof) {
    +        try {
    +          ByteBuffer buff;
    +          synchronized(client) {
    +            buff = client.getClient().downloadBlobChunk(beginBlobDownload.get_session());
    +          }
    +          buffer = buff.array();
    +          offset = buff.arrayOffset() + buff.position();
    +          int length = buff.remaining();
    +          end = offset + length;
    +          if (length == 0) {
    +            eof = true;
    +          }
    +        } catch (TException e) {
    +          throw new RuntimeException(e);
    +        }
    +      }
    +    }
    +    
    +    @Override 
    +    public synchronized int read(byte[] b) throws IOException {
    +      return read(b, 0, b.length);
    +    }
    +    
    +    @Override
    +    public synchronized int available() {
    +      return buffer == null ? 0 : (end - offset);
    +    }
    +
    +    @Override
    +    public long getFileLength() {
    +        return beginBlobDownload.get_data_size();
    +    }
    +  }
    +
    +  public class NimbusUploadAtomicOutputStream extends AtomicOutputStream {
    +    private String session;
    +    private int maxChunkSize = 4096;
    +    private String key;
    +
    +    public NimbusUploadAtomicOutputStream(String session, int bufferSize, String key) {
    +      this.session = session;
    +      this.maxChunkSize = bufferSize;
    +      this.key = key;
    +    }
    +
    +    @Override
    +    public void cancel() throws IOException {
    +      try {
    +        synchronized(client) {
    +          client.getClient().cancelBlobUpload(session);
    +        }
    +      } catch (TException e) {
    +        throw new RuntimeException(e);
    +      }
    +    }
    +
    +    @Override
    +    public void write(int b) throws IOException {
    +      try {
    +        synchronized(client) {
    +          client.getClient().uploadBlobChunk(session, ByteBuffer.wrap(new byte[] {(byte)b}));
    +        }
    +      } catch (TException e) {
    +        throw new RuntimeException(e);
    +      }
    +    }
    +    
    +    @Override
    +    public void write(byte []b) throws IOException {
    +      write(b, 0, b.length);
    +    }
    +    
    +    @Override
    +    public void write(byte []b, int offset, int len) throws IOException {
    +      try {
    +        int end = offset + len;
    +        for (int realOffset = offset; realOffset < end; realOffset += maxChunkSize) {
    +          int realLen = Math.min(end - realOffset, maxChunkSize);
    +          LOG.debug("Writing {} bytes of {} remaining",realLen,(end-realOffset));
    +          synchronized(client) {
    +            client.getClient().uploadBlobChunk(session, ByteBuffer.wrap(b, realOffset, realLen));
    +          }
    +        }
    +      } catch (TException e) {
    +        throw new RuntimeException(e);
    +      }
    +    }
    +    
    +    @Override
    +    public void close() throws IOException {
    +      try {
    +        synchronized(client) {
    +          client.getClient().finishBlobUpload(session);
    +          client.getClient().createStateInZookeeper(key);
    +        }
    +      } catch (TException e) {
    +        throw new RuntimeException(e);
    +      }
    +    }
    +  }
    +
    +  private NimbusClient client;
    +  private int bufferSize = 4096;
    +
    +  @Override
    +  public void prepare(Map conf) {
    +    this.client = NimbusClient.getConfiguredClient(conf);
    +    if (conf != null) {
    +        this.bufferSize = Utils.getInt(conf.get(Config.STORM_BLOBSTORE_INPUTSTREAM_BUFFER_SIZE_BYTES), bufferSize);
    +    }
    +  }
    +
    +  @Override
    +  protected AtomicOutputStream createBlobToExtend(String key, SettableBlobMeta meta)
    +      throws AuthorizationException, KeyAlreadyExistsException {
    +    try {
    +      synchronized(client) {
    +        return new NimbusUploadAtomicOutputStream(client.getClient().beginCreateBlob(key, meta), this.bufferSize, key);
    +      }
    +    } catch (AuthorizationException | KeyAlreadyExistsException exp) {
    +      throw exp;
    +    } catch (TException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override
    +  public AtomicOutputStream updateBlob(String key)
    +      throws AuthorizationException, KeyNotFoundException {
    +    try {
    +      synchronized(client) {
    +        return new NimbusUploadAtomicOutputStream(client.getClient().beginUpdateBlob(key), this.bufferSize, key);
    +      }
    +    } catch (AuthorizationException | KeyNotFoundException exp) {
    +      throw exp;
    +    } catch (TException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override
    +  public ReadableBlobMeta getBlobMeta(String key) throws AuthorizationException, KeyNotFoundException {
    +    try {
    +      synchronized(client) {
    +        return client.getClient().getBlobMeta(key);
    +      }
    +    } catch (AuthorizationException | KeyNotFoundException exp) {
    +      throw exp;
    +    } catch (TException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override
    +  protected void setBlobMetaToExtend(String key, SettableBlobMeta meta)
    +      throws AuthorizationException, KeyNotFoundException {
    +    try {
    +      synchronized(client) {
    +        client.getClient().setBlobMeta(key, meta);
    +      }
    +    } catch (AuthorizationException | KeyNotFoundException exp) {
    +      throw exp;
    +    } catch (TException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override
    +  public void deleteBlob(String key) throws AuthorizationException, KeyNotFoundException {
    +    try {
    +      synchronized(client) {
    +        client.getClient().deleteBlob(key);
    +      }
    +    } catch (AuthorizationException | KeyNotFoundException exp) {
    +      throw exp;
    +    } catch (TException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override
    +  public void createStateInZookeeper(String key) {
    +    try {
    +      synchronized(client) {
    +        client.getClient().createStateInZookeeper(key);
    +      }
    +    } catch (TException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override
    +  public InputStreamWithMeta getBlob(String key) throws AuthorizationException, KeyNotFoundException {
    +    try {
    +      synchronized(client) {
    +        return new NimbusDownloadInputStream(client.getClient().beginBlobDownload(key));
    +      }
    +    } catch (AuthorizationException | KeyNotFoundException exp) {
    +      throw exp;
    +    } catch (TException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override
    +  public Iterator<String> listKeys() {
    +    try {
    +      synchronized(client) {
    +        return new NimbusKeyIterator(client.getClient().listBlobs(""));
    +      }
    +    } catch (TException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override
    +  public int getBlobReplication(String key) throws AuthorizationException, KeyNotFoundException {
    +    try {
    +      return client.getClient().getBlobReplication(key);
    +    } catch (AuthorizationException | KeyNotFoundException exp) {
    +      throw exp;
    +    } catch (TException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override
    +  public int updateBlobReplication(String key, int replication) throws AuthorizationException, KeyNotFoundException {
    +    try {
    +      return client.getClient().updateBlobReplication(key, replication);
    +    } catch (AuthorizationException | KeyNotFoundException exp) {
    +      throw exp;
    +    } catch (TException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override
    +  public boolean setClient(Map conf, NimbusClient client) {
    +    this.client = client;
    +    if (conf != null) {
    +      this.bufferSize = Utils.getInt(conf.get(Config.STORM_BLOBSTORE_INPUTSTREAM_BUFFER_SIZE_BYTES), bufferSize);
    +    }
    +    return true;
    +  }
    +
    +  @Override
    +  protected void finalize() {
    --- End diff --
    
    It relieves only a client object before gc, might not be a big issue but no harm in keeping it I think.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45121400
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -1345,10 +1482,11 @@
                     principal (.principal req)
                     submitter-principal (if principal (.toString principal))
                     submitter-user (.toLocal principal-to-local principal)
    +                system-user (. System (getProperty "user.name"))
    --- End diff --
    
    ```Clojure
    system-user (System/getProperty "user.name")
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by unsleepy22 <gi...@git.apache.org>.
Github user unsleepy22 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46019946
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStore.java ---
    @@ -0,0 +1,444 @@
    +/**
    + * 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 java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Set;
    +import java.util.regex.Pattern;
    +
    +import javax.security.auth.Subject;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import backtype.storm.daemon.Shutdownable;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +
    +/**
    + * Provides a way to store blobs that can be downloaded.
    + * Blobs must be able to be uploaded and listed from Nimbus,
    + * and downloaded from the Supervisors. It is a key value based
    + * store. Key being a string and value being the blob data.
    + *
    + * ACL checking must take place against the provided subject.
    + * If the blob store does not support Security it must validate
    + * that all ACLs set are always WORLD, everything.
    + *
    + * The users can upload their blobs through the blob store command
    + * line. The command line also allows us to update and delete blobs.
    + *
    + * Modifying the replication factor only works for HdfsBlobStore
    + * as for the LocalFsBlobStore the replication is dependent on
    + * the number of Nimbodes available.
    + */
    +public abstract class BlobStore implements Shutdownable {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStore.class);
    --- End diff --
    
    why public?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by kishorvpatil <gi...@git.apache.org>.
Github user kishorvpatil commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43813266
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -1508,11 +1644,9 @@
                                                      (.set_assigned_memonheap topo-summ (get resources 3))
                                                      (.set_assigned_memoffheap topo-summ (get resources 4))
                                                      (.set_assigned_cpu topo-summ (get resources 5)))
    -                                               (.set_replication_count topo-summ (if (:code-distributor nimbus)
    -                                                                                   (.getReplicationCount (:code-distributor nimbus) id)
    -                                                                                   1))
    +                                               (.set_replication_count topo-summ (get-blob-replication-count (master-stormcode-key id) nimbus))
    --- End diff --
    
    spacing.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by unsleepy22 <gi...@git.apache.org>.
Github user unsleepy22 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46019849
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStore.java ---
    @@ -0,0 +1,444 @@
    +/**
    + * 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 java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Set;
    +import java.util.regex.Pattern;
    +
    +import javax.security.auth.Subject;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import backtype.storm.daemon.Shutdownable;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +
    +/**
    + * Provides a way to store blobs that can be downloaded.
    + * Blobs must be able to be uploaded and listed from Nimbus,
    + * and downloaded from the Supervisors. It is a key value based
    + * store. Key being a string and value being the blob data.
    + *
    + * ACL checking must take place against the provided subject.
    + * If the blob store does not support Security it must validate
    + * that all ACLs set are always WORLD, everything.
    + *
    + * The users can upload their blobs through the blob store command
    + * line. The command line also allows us to update and delete blobs.
    + *
    + * Modifying the replication factor only works for HdfsBlobStore
    + * as for the LocalFsBlobStore the replication is dependent on
    + * the number of Nimbodes available.
    + */
    +public abstract class BlobStore implements Shutdownable {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStore.class);
    +  private static final Pattern KEY_PATTERN = Pattern.compile("^[\\w \\t\\.:_-]+$");
    +  protected static final String BASE_BLOBS_DIR_NAME = "blobs";
    +
    +  /**
    +   * Allows us to initialize the blob store
    +   * @param conf The storm configuration
    +   * @param baseDir The directory path to store the blobs
    +   * @param nimbusInfo Contains the nimbus host, port and leadership information.
    +   */
    +  public abstract void prepare(Map conf, String baseDir, NimbusInfo nimbusInfo);
    +
    +  /**
    +   * Creates the blob.
    +   * @param key Key for the blob.
    +   * @param meta Metadata which contains the acls information
    +   * @param who Is the subject creating the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written.
    +   * @throws AuthorizationException
    +   * @throws KeyAlreadyExistsException
    +   */
    +  public abstract AtomicOutputStream createBlob(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException;
    +
    +  /**
    +   * Updates the blob data.
    +   * @param key Key for the blob.
    +   * @param who Is the subject having the write privilege for the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract AtomicOutputStream updateBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Gets the current version of metadata for a blob
    +   * to be viewed by the user or downloaded by the supervisor.
    +   * @param key Key for the blob.
    +   * @param who Is the subject having the read privilege for the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract ReadableBlobMeta getBlobMeta(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Sets the metadata with renewed acls for the blob.
    +   * @param key Key for the blob.
    +   * @param meta Metadata which contains the updated
    +   * acls information.
    +   * @param who Is the subject having the write privilege for the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract void setBlobMeta(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Deletes the blob data and metadata.
    +   * @param key Key for the blob.
    +   * @param who Is the subject having write privilege for the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract void deleteBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Gets the InputStream to read the blob details
    +   * @param key Key for the blob.
    +   * @param who Is the subject having the read privilege for the blob.
    +   * @return InputStreamWithMeta has the additional
    +   * file length and version information.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract InputStreamWithMeta getBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Returns an iterator with all the list of
    +   * keys currently available on the blob store.
    +   * @return Iterator<String>
    +   */
    +  public abstract Iterator<String> listKeys();
    +
    +  /**
    +   * Gets the replication factor of the blob.
    +   * @param key Key for the blob.
    +   * @param who Is the subject having the read privilege for the blob.
    +   * @return BlobReplication object containing the
    +   * replication factor for the blob.
    +   * @throws Exception
    +   */
    +  public abstract int getBlobReplication(String key, Subject who) throws Exception;
    +
    +  /**
    +   * Modifies the replication factor of the blob.
    +   * @param key Key for the blob.
    +   * @param replication The replication factor the
    +   * blob has to be set.
    +   * @param who Is the subject having the update privilege for the blob
    +   * @return BlobReplication object containing the
    +   * updated replication factor for the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   * @throws IOException
    +   */
    +  public abstract int updateBlobReplication(String key, int replication, Subject who) throws AuthorizationException, KeyNotFoundException, IOException;
    +
    +  /**
    +   * Filters keys based on the KeyFilter
    +   * passed as the argument.
    +   * @param filter KeyFilter
    +   * @param <R> Type
    +   * @return Set of filtered keys
    +   */
    +  public <R> Set<R> filterAndListKeys(KeyFilter<R> filter) {
    +    Set<R> ret = new HashSet<R>();
    +    Iterator<String> keys = listKeys();
    +    while (keys.hasNext()) {
    +      String key = keys.next();
    +      R filtered = filter.filter(key);
    +      if (filtered != null) {
    +        ret.add(filtered);
    +      }
    +    }
    +    return ret;
    +  }
    +
    +  /**
    +   * Validates key checking for potentially harmful patterns
    +   * @param key Key for the blob.
    +   */
    +  public static final void validateKey(String key) {
    +    if (key == null || key.isEmpty() || "..".equals(key) || ".".equals(key) || !KEY_PATTERN.matcher(key).matches()) {
    --- End diff --
    
    use StringUtils.isEmpty from apache-commons-lang?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the pull request:

    https://github.com/apache/storm/pull/845#issuecomment-161798173
  
    @ptgoetz Thanks for the review.  I agree with the follow on items.  Because this is such a big change and @redsanket has been upmerging this regularly I would like to see this merged in sooner rather then later.  I'll try and merge this in tomorrow assuming no one else has any issues with it between now and then.  The other work can be done in follow on JIRA and I will update the 1.0 wiki to include the new JIRA once they are filed.
    
    Thank you everyone for your help with getting this in. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45433271
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/KeyVersion.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * 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.zookeeper.CreateMode;
    +import org.apache.zookeeper.ZooDefs;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.TreeSet;
    +import java.util.Map;
    +import java.util.List;
    +
    +/**
    + * Class hands over the version of the key to be stored within the zookeeper
    + */
    +public class KeyVersion {
    +  private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
    +  private final String BLOBSTORE_SUBTREE="/blobstore";
    +  private final String BLOBSTORE_KEY_COUNTER_SUBTREE="/blobstorekeycounter";
    +  private String key;
    +  private NimbusInfo nimbusInfo;
    +
    +  public KeyVersion(String key, NimbusInfo nimbusInfo) {
    +    this.key = key;
    +    this.nimbusInfo = nimbusInfo;
    +  }
    +
    +  public int getKeyVersion(Map conf) {
    +    TreeSet<Integer> versions = new TreeSet<Integer>();
    +    CuratorFramework zkClient = Utils.createZKClient(conf);
    +    try {
    +      // Key has not been created yet and it is the first time it is being created
    +      if(zkClient.checkExists().forPath(BLOBSTORE_SUBTREE + "/" + key) == null) {
    +        zkClient.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT)
    +                .withACL(ZooDefs.Ids.OPEN_ACL_UNSAFE).forPath(BLOBSTORE_KEY_COUNTER_SUBTREE + "/" + key + "/" + 1);
    +        return 1;
    +      }
    +
    +      // When all nimbodes go down and one or few of them come up
    +      // Unfortunately there might not be an exact way to know which one contains the most updated blob
    +      // if all go down which is unlikely. Hence there might be a need to update the blob if all go down
    +      List<String> stateInfoList = zkClient.getChildren().forPath(BLOBSTORE_SUBTREE + "/" + key);
    +      LOG.debug("stateInfoList {} stateInfoList {}", stateInfoList.size(), stateInfoList);
    +      if(stateInfoList.isEmpty()) {
    +        return getKeyVersionCounterValue(zkClient, key);
    +      }
    +
    +      LOG.debug("stateInfoSize {}", stateInfoList.size());
    +      // In all other cases check for the latest version on the nimbus and assign the version
    +      // check if all are have same version, if not assign the highest version
    +      for (String stateInfo:stateInfoList) {
    +        versions.add(Integer.parseInt(Utils.normalizeVersionInfo(stateInfo)[1]));
    +      }
    +
    +      int currentCounter = getKeyVersionCounterValue(zkClient, key);
    +      // This condition returns version when a nimbus crashes and comes up
    +      if (!checkIfStateContainsCurrentNimbusHost(stateInfoList, nimbusInfo) && !nimbusInfo.isLeader()) {
    +        if (versions.last() < currentCounter) {
    +          return currentCounter;
    +        } else {
    +          return currentCounter - 1;
    +        }
    +      }
    +      // Condition checks for an update scenario
    +      if (stateInfoList.size() >= 1 && versions.size() == 1) {
    --- End diff --
    
    I will document it properly and explain my thought process


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45133934
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -454,10 +455,82 @@
           (shutdown-worker supervisor id))
         ))
     
    +(defn get-blob-localname
    +  "Given the blob information either gets the localname field if it exists,
    +  else routines the default value passed in."
    +  [blob-info defaultValue]
    +  (if-let [val (if blob-info (get blob-info "localname") nil)] val defaultValue))
    --- End diff --
    
    ```Clojure
    (or (get blob-info "localname") defaultValue)
    ```
    The `get` returns `nil` if `blob-info` is `nil`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43773503
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -14,15 +14,29 @@
     ;; See the License for the specific language governing permissions and
     ;; limitations under the License.
     (ns backtype.storm.daemon.nimbus
    -  (:import [org.apache.thrift.server THsHaServer THsHaServer$Args])
    +  (:import [org.apache.thrift.server THsHaServer THsHaServer$Args]
    +           [backtype.storm.generated KeyNotFoundException]
    +           [backtype.storm.blobstore LocalFsBlobStore])
    --- End diff --
    
    Can we split these up to be separate `(:import ` calls per line?  I know we do it in other places in the code but for consistency it would be nice to only do it one per line.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43769317
  
    --- Diff: storm-core/pom.xml ---
    @@ -344,6 +431,9 @@
                         <sourceDirectories>
                             <sourceDirectory>src/clj</sourceDirectory>
                         </sourceDirectories>
    +                    <namespaces>
    +                        <namespace>backtype.storm.daemon.*</namespace>
    +                    </namespaces>
    --- End diff --
    
    What does this do?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43770063
  
    --- Diff: storm-core/src/clj/backtype/storm/cluster.clj ---
    @@ -248,9 +256,9 @@
       [id]
       (str ASSIGNMENTS-SUBTREE "/" id))
     
    -(defn code-distributor-path
    -  [id]
    -  (str CODE-DISTRIBUTOR-SUBTREE "/" id))
    +(defn blobstore-path
    +  [key]
    +  (str BLOBSTORE-SUBTREE "/" key))
    --- End diff --
    
    This makes me a bit nervous, as key is a user provided value, and if we put in a `..` or a `/` or something else malicious it could cause a security issue.  Are we checking elsewhere to prevent the key from having potentially harmful characters in it?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by kishorvpatil <gi...@git.apache.org>.
Github user kishorvpatil commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43821520
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/HdfsClientBlobStore.java ---
    @@ -0,0 +1,105 @@
    +package backtype.storm.blobstore;
    +
    +import backtype.storm.Config;
    +import backtype.storm.generated.*;
    --- End diff --
    
    explicit import


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43774601
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -346,53 +394,124 @@
           [(.getNodeId slot) (.getPort slot)]
           )))
     
    +(defn- get-nimbus-subject []
    +  (let [nimbus-subject (Subject.)
    +        nimbus-principal (NimbusPrincipal.)
    +        principals (.getPrincipals nimbus-subject)]
    +    (.add principals nimbus-principal)
    +    nimbus-subject))
    +
    +(defn- get-metadata-version [blob-store key subject]
    +  (let [blob-meta (.getBlobMeta blob-store key subject)]
    +    (.get_version blob-meta)))
    +
    +(defn get-key-list-from-blob-store [blob-store]
    +  (let [key-iter (.listKeys blob-store (get-nimbus-subject))
    +        keys (iterator-seq key-iter)]
    +    (if (not-nil? keys)
    +      (java.util.ArrayList. keys)
    +      [])))
    +
     (defn- setup-storm-code [nimbus conf storm-id tmp-jar-location storm-conf topology]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -   (log-message "nimbus file location:" stormroot)
    -   (FileUtils/forceMkdir (File. stormroot))
    -   (FileUtils/cleanDirectory (File. stormroot))
    -   (setup-jar conf tmp-jar-location stormroot)
    -   (FileUtils/writeByteArrayToFile (File. (master-stormcode-path stormroot)) (Utils/serialize topology))
    -   (FileUtils/writeByteArrayToFile (File. (master-stormconf-path stormroot)) (Utils/toCompressedJsonConf storm-conf))
    -   (if (:code-distributor nimbus) (.upload (:code-distributor nimbus) stormroot storm-id))
    -   ))
    +  (let [subject (get-subject)
    +        storm-cluster-state (:storm-cluster-state nimbus)
    +        blob-store (:blob-store nimbus)
    +        jar-key (master-stormjar-key storm-id)
    +        code-key (master-stormcode-key storm-id)
    +        conf-key (master-stormconf-key storm-id)
    +        nimbus-host-port-info (:nimbus-host-port-info nimbus)]
    +    (log-message "subject-changed" subject)
    --- End diff --
    
    Can we make some of these debug statements instead of info, or possibly remove them?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by unsleepy22 <gi...@git.apache.org>.
Github user unsleepy22 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46024323
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStore.java ---
    @@ -0,0 +1,296 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.utils.Utils;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.ByteArrayOutputStream;
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.FileNotFoundException;
    +import java.io.InputStream;
    +
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;;
    +
    +import static backtype.storm.blobstore.BlobStoreAclHandler.ADMIN;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.READ;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.WRITE;
    +
    +/**
    + * Provides a local file system backed blob store implementation for Nimbus.
    + */
    +public class LocalFsBlobStore extends BlobStore {
    +  public static final Logger LOG = LoggerFactory.getLogger(LocalFsBlobStore.class);
    +  private static final String DATA_PREFIX = "data_";
    +  private static final String META_PREFIX = "meta_";
    +  protected BlobStoreAclHandler _aclHandler;
    +  private final String BLOBSTORE_SUBTREE = "/blobstore/";
    +  private NimbusInfo nimbusInfo;
    +  private FileBlobStoreImpl fbs;
    +  private Map conf;
    +
    +  @Override
    +  public void prepare(Map conf, String overrideBase, NimbusInfo nimbusInfo) {
    +    this.conf = conf;
    +    this.nimbusInfo = nimbusInfo;
    +     if (overrideBase == null) {
    +       overrideBase = (String)conf.get(Config.BLOBSTORE_DIR);
    +      if (overrideBase == null) {
    +        overrideBase = (String) conf.get(Config.STORM_LOCAL_DIR);
    +      }
    +    }
    +    File baseDir = new File(overrideBase, BASE_BLOBS_DIR_NAME);
    +    try {
    +      fbs = new FileBlobStoreImpl(baseDir, conf);
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +    _aclHandler = new BlobStoreAclHandler(conf);
    +  }
    +
    +  @Override
    +  public AtomicOutputStream createBlob(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException {
    +    LOG.debug("Creating Blob for key {}", key);
    +    validateKey(key);
    +    _aclHandler.normalizeSettableBlobMeta(key, meta, who, READ | WRITE | ADMIN);
    +    BlobStoreAclHandler.validateSettableACLs(key, meta.get_acl());
    +    _aclHandler.hasPermissions(meta.get_acl(), READ | WRITE | ADMIN, who, key);
    +    if (fbs.exists(DATA_PREFIX+key)) {
    +      throw new KeyAlreadyExistsException(key);
    +    }
    +    BlobStoreFileOutputStream mOut = null;
    +    try {
    +      mOut = new BlobStoreFileOutputStream(fbs.write(META_PREFIX+key, true));
    +      mOut.write(Utils.thriftSerialize(meta));
    +      mOut.close();
    +      mOut = null;
    +      return new BlobStoreFileOutputStream(fbs.write(DATA_PREFIX+key, true));
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    } finally {
    +      if (mOut != null) {
    +        try {
    +          mOut.cancel();
    +        } catch (IOException e) {
    +          //Ignored
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public AtomicOutputStream updateBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    validateKey(key);
    +    checkForBlobOrDownload(key);
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(meta.get_acl(), WRITE, who, key);
    +    try {
    +      return new BlobStoreFileOutputStream(fbs.write(DATA_PREFIX+key, false));
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  private SettableBlobMeta getStoredBlobMeta(String key) throws KeyNotFoundException {
    +    InputStream in = null;
    +    try {
    +      LocalFsBlobStoreFile pf = fbs.read(META_PREFIX+key);
    +      try {
    +        in = pf.getInputStream();
    +      } catch (FileNotFoundException fnf) {
    +        throw new KeyNotFoundException(key);
    +      }
    +      ByteArrayOutputStream out = new ByteArrayOutputStream();
    +      byte [] buffer = new byte[2048];
    +      int len;
    +      while ((len = in.read(buffer)) > 0) {
    +        out.write(buffer, 0, len);
    +      }
    +      in.close();
    +      in = null;
    +      return Utils.thriftDeserialize(SettableBlobMeta.class, out.toByteArray());
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    } finally {
    +      if (in != null) {
    +        try {
    +          in.close();
    +        } catch (IOException e) {
    +          //Ignored
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public ReadableBlobMeta getBlobMeta(String key, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    validateKey(key);
    +    if(!checkForBlobOrDownload(key)) {
    +      checkForBlobUpdate(key);
    +    }
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.validateUserCanReadMeta(meta.get_acl(), who, key);
    +    ReadableBlobMeta rbm = new ReadableBlobMeta();
    +    rbm.set_settable(meta);
    +    try {
    +      LocalFsBlobStoreFile pf = fbs.read(DATA_PREFIX+key);
    +      rbm.set_version(pf.getModTime());
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +    return rbm;
    +  }
    +
    +  @Override
    +  public void setBlobMeta(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    validateKey(key);
    +    checkForBlobOrDownload(key);
    +    _aclHandler.normalizeSettableBlobMeta(key, meta, who, ADMIN);
    +    BlobStoreAclHandler.validateSettableACLs(key, meta.get_acl());
    +    SettableBlobMeta orig = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(orig.get_acl(), ADMIN, who, key);
    +    BlobStoreFileOutputStream mOut = null;
    +    try {
    +      mOut = new BlobStoreFileOutputStream(fbs.write(META_PREFIX+key, false));
    +      mOut.write(Utils.thriftSerialize(meta));
    +      mOut.close();
    +      mOut = null;
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    } finally {
    +      if (mOut != null) {
    +        try {
    +          mOut.cancel();
    +        } catch (IOException e) {
    +          //Ignored
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void deleteBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    validateKey(key);
    +    checkForBlobOrDownload(key);
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(meta.get_acl(), WRITE, who, key);
    +    try {
    +      fbs.deleteKey(DATA_PREFIX+key);
    +      fbs.deleteKey(META_PREFIX+key);
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override
    +  public InputStreamWithMeta getBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    validateKey(key);
    +    if(!checkForBlobOrDownload(key)) {
    +      checkForBlobUpdate(key);
    +    }
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(meta.get_acl(), READ, who, key);
    +    try {
    +      return new BlobStoreFileInputStream(fbs.read(DATA_PREFIX+key));
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override
    +  public Iterator<String> listKeys() {
    +    try {
    +      return new KeyTranslationIterator(fbs.listKeys(), DATA_PREFIX);
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override
    +  public void shutdown() {
    +  }
    +
    +  @Override
    +  public int getBlobReplication(String key, Subject who) throws Exception {
    +    validateKey(key);
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(meta.get_acl(), READ, who, key);
    +    CuratorFramework zkClient = BlobStoreUtils.createZKClient(conf);
    +    if (zkClient.checkExists().forPath(BLOBSTORE_SUBTREE + key) == null) {
    +       zkClient.close();
    +       return 0;
    +    }
    +    int replicationCount = zkClient.getChildren().forPath(BLOBSTORE_SUBTREE + key).size();
    +    zkClient.close();
    +    return replicationCount;
    +  }
    +
    +  @Override
    +  public int updateBlobReplication(String key, int replication, Subject who) throws AuthorizationException, KeyNotFoundException {
    +      throw new UnsupportedOperationException("For local file system blob store the update blobs function does not work. " +
    +              "Please use HDFS blob store to make this feature available.");
    +  }
    +
    +  //This additional check and download is for nimbus high availability in case you have more than one nimbus
    +  public boolean checkForBlobOrDownload(String key) {
    +    boolean checkBlobDownload = false;
    +    try {
    +      List<String> keyList = BlobStoreUtils.getKeyListFromBlobStore(this);
    +      if (!keyList.contains(key)) {
    +        CuratorFramework zkClient = BlobStoreUtils.createZKClient(conf);
    +        if (zkClient.checkExists().forPath(BLOBSTORE_SUBTREE + key) != null) {
    +          Set<NimbusInfo> nimbusSet = BlobStoreUtils.getNimbodesWithLatestSequenceNumberOfBlob(zkClient, key);
    +          if (BlobStoreUtils.downloadMissingBlob(conf, this, key, nimbusSet)) {
    +            LOG.debug("Updating blobs state");
    +              BlobStoreUtils.createStateInZookeeper(conf, key, nimbusInfo);
    +            checkBlobDownload = true;
    +          }
    +        }
    +        zkClient.close();
    --- End diff --
    
    same as above, place `zkClient.close` in finally bock, same for all below if any.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43765581
  
    --- Diff: conf/defaults.yaml ---
    @@ -112,6 +112,20 @@ transactional.zookeeper.root: "/transactional"
     transactional.zookeeper.servers: null
     transactional.zookeeper.port: null
     
    +## blobstore configs
    +supervisor.blobstore.class: "backtype.storm.blobstore.NimbusBlobStore"
    +supervisor.blobstore.download.thread.count: 5
    +supervisor.blobstore.download.max_retries: 3
    +supervisor.localizer.cache.target.size.mb: 10240
    +supervisor.localizer.cleanup.interval.ms: 600000
    +
    +nimbus.blobstore.class: "backtype.storm.blobstore.LocalFsBlobStore"
    +nimbus.blobstore.expiration.secs: 600
    +
    +storm.blobstore.inputstream.buffer.size.bytes: 65536
    +client.blobstore.class: "backtype.storm.blobstore.NimbusBlobStore"
    +blobstore.replication.factor: 3
    --- End diff --
    
    The naming of some of these configs seems inconsistent.  Should `blobstore.replication.factor` be `storm.blobstore.replication.factor`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45428570
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/KeyVersion.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * 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.zookeeper.CreateMode;
    +import org.apache.zookeeper.ZooDefs;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.TreeSet;
    +import java.util.Map;
    +import java.util.List;
    +
    +/**
    + * Class hands over the version of the key to be stored within the zookeeper
    + */
    +public class KeyVersion {
    +  private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
    +  private final String BLOBSTORE_SUBTREE="/blobstore";
    +  private final String BLOBSTORE_KEY_COUNTER_SUBTREE="/blobstorekeycounter";
    +  private String key;
    +  private NimbusInfo nimbusInfo;
    +
    +  public KeyVersion(String key, NimbusInfo nimbusInfo) {
    +    this.key = key;
    +    this.nimbusInfo = nimbusInfo;
    +  }
    +
    +  public int getKeyVersion(Map conf) {
    +    TreeSet<Integer> versions = new TreeSet<Integer>();
    +    CuratorFramework zkClient = Utils.createZKClient(conf);
    +    try {
    +      // Key has not been created yet and it is the first time it is being created
    +      if(zkClient.checkExists().forPath(BLOBSTORE_SUBTREE + "/" + key) == null) {
    +        zkClient.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT)
    +                .withACL(ZooDefs.Ids.OPEN_ACL_UNSAFE).forPath(BLOBSTORE_KEY_COUNTER_SUBTREE + "/" + key + "/" + 1);
    +        return 1;
    +      }
    +
    +      // When all nimbodes go down and one or few of them come up
    +      // Unfortunately there might not be an exact way to know which one contains the most updated blob
    +      // if all go down which is unlikely. Hence there might be a need to update the blob if all go down
    +      List<String> stateInfoList = zkClient.getChildren().forPath(BLOBSTORE_SUBTREE + "/" + key);
    +      LOG.debug("stateInfoList {} stateInfoList {}", stateInfoList.size(), stateInfoList);
    +      if(stateInfoList.isEmpty()) {
    +        return getKeyVersionCounterValue(zkClient, key);
    +      }
    +
    +      LOG.debug("stateInfoSize {}", stateInfoList.size());
    +      // In all other cases check for the latest version on the nimbus and assign the version
    +      // check if all are have same version, if not assign the highest version
    +      for (String stateInfo:stateInfoList) {
    +        versions.add(Integer.parseInt(Utils.normalizeVersionInfo(stateInfo)[1]));
    +      }
    +
    +      int currentCounter = getKeyVersionCounterValue(zkClient, key);
    +      // This condition returns version when a nimbus crashes and comes up
    +      if (!checkIfStateContainsCurrentNimbusHost(stateInfoList, nimbusInfo) && !nimbusInfo.isLeader()) {
    +        if (versions.last() < currentCounter) {
    +          return currentCounter;
    +        } else {
    +          return currentCounter - 1;
    --- End diff --
    
    Could we add a comment explaining why we want to return `currentCounter - 1` here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45264351
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -732,25 +854,90 @@
      (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
       (worker-launcher-and-wait conf (storm-conf TOPOLOGY-SUBMITTER-USER) ["code-dir" dir] :log-prefix (str "setup conf for " dir))))
     
    +(defn setup-blob-permission [conf storm-conf path]
    +  (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
    +    (worker-launcher-and-wait conf (storm-conf TOPOLOGY-SUBMITTER-USER) ["blob" path] :log-prefix (str "setup blob permissions for " path))))
    +
    +(defn setup-storm-code-dir [conf storm-conf dir]
    +  (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
    +    (worker-launcher-and-wait conf (storm-conf TOPOLOGY-SUBMITTER-USER) ["code-dir" dir] :log-prefix (str "setup conf for " dir))))
    +
    +(defn download-blobs-for-topology!
    +  "Download all blobs listed in the topology configuration for a given topology."
    +  [conf stormconf-path localizer tmproot]
    +  (let [storm-conf (read-supervisor-storm-conf-given-path conf stormconf-path)
    +        blobstore-map (storm-conf TOPOLOGY-BLOBSTORE-MAP)
    +        user (storm-conf TOPOLOGY-SUBMITTER-USER)
    +        topo-name (storm-conf TOPOLOGY-NAME)
    +        user-dir (.getLocalUserFileCacheDir localizer user)
    +        localresources (blobstore-map-to-localresources blobstore-map)]
    +    (when localresources
    +      (when-not (.exists user-dir)
    +        (FileUtils/forceMkdir user-dir)
    +        (setup-blob-permission conf storm-conf (.toString user-dir)))
    +      (try
    +        (let [localized-resources (.getBlobs localizer localresources user topo-name user-dir)]
    +          (setup-blob-permission conf storm-conf (.toString user-dir))
    +          (doseq [local-rsrc localized-resources]
    +            (let [rsrc-file-path (File. (.getFilePath local-rsrc))
    +                  key-name (.getName rsrc-file-path)
    +                  blob-symlink-target-name (.getName (File. (.getCurrentSymlinkPath local-rsrc)))
    +                  symlink-name (get-blob-localname (get blobstore-map key-name) key-name)]
    +              (create-symlink! tmproot (.getParent rsrc-file-path) symlink-name
    +                blob-symlink-target-name))))
    +        (catch AuthorizationException authExp
    +          (log-error authExp))
    +        (catch KeyNotFoundException knf
    +          (log-error knf))))))
    +
    +(defn get-blob-file-names
    +  [blobstore-map]
    +  (if blobstore-map
    +    (for [[k, data] blobstore-map]
    +      (get-blob-localname data k))))
    +
    +(defn download-blobs-for-topology-succeed?
    +  "Assert if all blobs are downloaded for the given topology"
    +  [stormconf-path target-dir]
    +  (let [storm-conf (clojurify-structure (Utils/fromCompressedJsonConf (FileUtils/readFileToByteArray (File. stormconf-path))))
    +        blobstore-map (storm-conf TOPOLOGY-BLOBSTORE-MAP)
    +        file-names (get-blob-file-names blobstore-map)]
    +    (if (and file-names (> (count file-names) 0))
    +      (every? #(Utils/checkFileExists target-dir %) file-names)
    +      true)))
    --- End diff --
    
    We can use `empty?` for both a `nil` list and an empty list:
    
    ```Clojure
    (if-not (empty? file-names)
      (every? #(Utils/checkFileExists target-dir %) file-names)
      true)
    ```
    
    or 
    
    ```Clojure
    (or (empty? file-names)
        (every? #(Utils/checkFileExists target-dir %) file-names))
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by unsleepy22 <gi...@git.apache.org>.
Github user unsleepy22 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46020162
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStoreAclHandler.java ---
    @@ -0,0 +1,401 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.AccessControl;
    +import backtype.storm.generated.AccessControlType;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.security.auth.AuthUtils;
    +import backtype.storm.security.auth.IPrincipalToLocal;
    +import backtype.storm.security.auth.NimbusPrincipal;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.security.Principal;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Provides common handling of acls for Blobstores.
    + * Also contains some static utility functions related to Blobstores.
    + */
    +public class BlobStoreAclHandler {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStoreAclHandler.class);
    +  private final IPrincipalToLocal _ptol;
    +
    +  public static final int READ = 0x01;
    +  public static final int WRITE = 0x02;
    +  public static final int ADMIN = 0x04;
    +  public static final List<AccessControl> WORLD_EVERYTHING =
    +      Arrays.asList(new AccessControl(AccessControlType.OTHER, READ | WRITE | ADMIN));
    +  public static final List<AccessControl> DEFAULT = new ArrayList<AccessControl>();
    +  private Set<String> _supervisors;
    +  private Set<String> _admins;
    +
    +  public BlobStoreAclHandler(Map conf) {
    +    _ptol = AuthUtils.GetPrincipalToLocalPlugin(conf);
    +    _supervisors = new HashSet<String>();
    +    _admins = new HashSet<String>();
    +    if (conf.containsKey(Config.NIMBUS_SUPERVISOR_USERS)) {
    +      _supervisors.addAll((List<String>)conf.get(Config.NIMBUS_SUPERVISOR_USERS));
    +    }
    +    if (conf.containsKey(Config.NIMBUS_ADMINS)) {
    +      _admins.addAll((List<String>)conf.get(Config.NIMBUS_ADMINS));
    +    }
    +  }
    +
    +  private static AccessControlType parseACLType(String type) {
    +    if ("other".equalsIgnoreCase(type) || "o".equalsIgnoreCase(type)) {
    +      return AccessControlType.OTHER;
    +    } else if ("user".equalsIgnoreCase(type) || "u".equalsIgnoreCase(type)) {
    +      return AccessControlType.USER;
    +    }
    +    throw new IllegalArgumentException(type+" is not a valid access control type");
    +  }
    +
    +  private static int parseAccess(String access) {
    +    int ret = 0;
    +    for (char c: access.toCharArray()) {
    +      if ('r' == c) {
    +        ret = ret | READ;
    +      } else if ('w' == c) {
    +        ret = ret | WRITE;
    +      } else if ('a' == c) {
    +        ret = ret | ADMIN;
    +      } else if ('-' == c) {
    +        //ignored
    +      } else {
    +        throw new IllegalArgumentException("");
    +      }
    +    }
    +    return ret;
    +  }
    +
    +  public static AccessControl parseAccessControl(String str) {
    +    String[] parts = str.split(":");
    +    String type = "other";
    +    String name = "";
    +    String access = "-";
    +    if (parts.length > 3) {
    +      throw new IllegalArgumentException("Don't know how to parse "+str+" into an ACL value");
    +    } else if (parts.length == 1) {
    +      type = "other";
    +      name = "";
    +      access = parts[0];
    +    } else if (parts.length == 2) {
    +      type = "user";
    +      name = parts[0];
    +      access = parts[1];
    +    } else if (parts.length == 3) {
    +      type = parts[0];
    +      name = parts[1];
    +      access = parts[2];
    +    }
    +    AccessControl ret = new AccessControl();
    +    ret.set_type(parseACLType(type));
    +    ret.set_name(name);
    +    ret.set_access(parseAccess(access));
    +    return ret;
    +  }
    +
    +  private static String accessToString(int access) {
    +    StringBuffer ret = new StringBuffer();
    +    ret.append(((access & READ) > 0) ? "r" : "-");
    +    ret.append(((access & WRITE) > 0) ? "w" : "-");
    +    ret.append(((access & ADMIN) > 0) ? "a" : "-");
    +    return ret.toString();
    +  }
    +
    +  public static String accessControlToString(AccessControl ac) {
    +    StringBuffer ret = new StringBuffer();
    --- End diff --
    
    use StringBuilder


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45760759
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/KeyVersion.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * 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.zookeeper.CreateMode;
    +import org.apache.zookeeper.ZooDefs;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.TreeSet;
    +import java.util.Map;
    +import java.util.List;
    +
    +/**
    + * Class hands over the version of the key to be stored within the zookeeper
    + */
    +public class KeyVersion {
    +  private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
    +  private final String BLOBSTORE_SUBTREE="/blobstore";
    +  private final String BLOBSTORE_KEY_COUNTER_SUBTREE="/blobstorekeycounter";
    +  private String key;
    +  private NimbusInfo nimbusInfo;
    +
    +  public KeyVersion(String key, NimbusInfo nimbusInfo) {
    +    this.key = key;
    +    this.nimbusInfo = nimbusInfo;
    +  }
    +
    +  public int getKeyVersion(Map conf) {
    +    TreeSet<Integer> versions = new TreeSet<Integer>();
    +    CuratorFramework zkClient = Utils.createZKClient(conf);
    +    try {
    +      // Key has not been created yet and it is the first time it is being created
    +      if(zkClient.checkExists().forPath(BLOBSTORE_SUBTREE + "/" + key) == null) {
    +        zkClient.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT)
    +                .withACL(ZooDefs.Ids.OPEN_ACL_UNSAFE).forPath(BLOBSTORE_KEY_COUNTER_SUBTREE + "/" + key + "/" + 1);
    +        return 1;
    +      }
    +
    +      // When all nimbodes go down and one or few of them come up
    +      // Unfortunately there might not be an exact way to know which one contains the most updated blob
    +      // if all go down which is unlikely. Hence there might be a need to update the blob if all go down
    +      List<String> stateInfoList = zkClient.getChildren().forPath(BLOBSTORE_SUBTREE + "/" + key);
    +      LOG.debug("stateInfoList {} stateInfoList {}", stateInfoList.size(), stateInfoList);
    +      if(stateInfoList.isEmpty()) {
    +        return getKeyVersionCounterValue(zkClient, key);
    +      }
    +
    +      LOG.debug("stateInfoSize {}", stateInfoList.size());
    +      // In all other cases check for the latest version on the nimbus and assign the version
    +      // check if all are have same version, if not assign the highest version
    +      for (String stateInfo:stateInfoList) {
    +        versions.add(Integer.parseInt(Utils.normalizeVersionInfo(stateInfo)[1]));
    +      }
    +
    +      int currentCounter = getKeyVersionCounterValue(zkClient, key);
    +      // This condition returns version when a nimbus crashes and comes up
    +      if (!checkIfStateContainsCurrentNimbusHost(stateInfoList, nimbusInfo) && !nimbusInfo.isLeader()) {
    +        if (versions.last() < currentCounter) {
    +          return currentCounter;
    +        } else {
    +          return currentCounter - 1;
    +        }
    +      }
    +      // Condition checks for an update scenario
    +      if (stateInfoList.size() >= 1 && versions.size() == 1) {
    +        if (versions.first() < getKeyVersionCounterValue(zkClient, key)) {
    +          incrementCounter(zkClient, key, currentCounter);
    +          return currentCounter + 1;
    +        } else {
    +          incrementCounter(zkClient, key, currentCounter);
    +          return versions.first() + 1;
    +        }
    +      }
    +    } catch(Exception e) {
    +      LOG.error("Exception {}", e);
    +    } finally {
    +      if (zkClient != null) {
    +        zkClient.close();
    +      }
    +    }
    +    return versions.last();
    +  }
    +
    +  public boolean checkIfStateContainsCurrentNimbusHost(List<String> stateInfoList, NimbusInfo nimbusInfo) {
    --- End diff --
    
    @d2r addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r44989851
  
    --- Diff: external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java ---
    @@ -0,0 +1,381 @@
    +/**
    + * 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 org.apache.storm.hdfs.blobstore;
    +
    +import backtype.storm.Config;
    +import backtype.storm.blobstore.AtomicOutputStream;
    +import backtype.storm.blobstore.AtomicOutputStream;
    +import backtype.storm.blobstore.BlobStore;
    +import backtype.storm.blobstore.BlobStoreAclHandler;
    +import backtype.storm.blobstore.BlobStoreFile;
    +import backtype.storm.blobstore.InputStreamWithMeta;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.utils.NimbusClient;
    +import backtype.storm.utils.Utils;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.security.UserGroupInformation;
    +import org.apache.thrift7.TBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.ByteArrayOutputStream;
    +import java.io.FileNotFoundException;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.security.AccessController;
    +import java.security.PrivilegedAction;
    +import java.util.Iterator;
    +import java.util.Map;
    +
    +import static backtype.storm.blobstore.BlobStoreAclHandler.ADMIN;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.READ;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.WRITE;
    +
    +/**
    + * Provides a HDFS file system backed blob store implementation.
    + * Note that this provides an api for having HDFS be the backing store for the blobstore,
    + * it is not a service/daemon.
    + */
    +public class HdfsBlobStore extends BlobStore {
    +  public static final Logger LOG = LoggerFactory.getLogger(HdfsBlobStore.class);
    +  private static final String DATA_PREFIX = "data_";
    +  private static final String META_PREFIX = "meta_";
    +  private BlobStoreAclHandler _aclHandler;
    +  private HdfsBlobStoreImpl _hbs;
    +  private Subject _localSubject;
    +  private Map conf;
    +
    +  /*
    +   * Get the subject from Hadoop so we can use it to validate the acls. There is no direct
    +   * interface from UserGroupInformation to get the subject, so do a doAs and get the context.
    +   * We could probably run everything in the doAs but for now just grab the subject.
    +   */
    +  private Subject getHadoopUser() {
    +    Subject subj;
    +    try {
    +      subj = UserGroupInformation.getCurrentUser().doAs(
    +          new PrivilegedAction<Subject>() {
    +            @Override
    +            public Subject run() {
    +              return Subject.getSubject(AccessController.getContext());
    +            }
    +          });
    +    } catch (IOException e) {
    +      throw new RuntimeException("Error creating subject and logging user in!", e);
    +    }
    +    return subj;
    +  }
    +
    +  // If who is null then we want to use the user hadoop says we are.
    +  // Required for the supervisor to call these routines as its not
    +  // logged in as anyone.
    +  private Subject checkAndGetSubject(Subject who) {
    +    if (who == null) {
    +      return _localSubject;
    +    }
    +    return who;
    +  }
    +
    +  @Override
    +  public void prepare(Map conf, String overrideBase, NimbusInfo nimbusInfo) {
    +    this.conf = conf;
    +    prepareInternal(conf, overrideBase, null);
    +  }
    +
    +  /*
    +   * Allow a Hadoop Configuration to be passed for testing. If it's null then the hadoop configs
    +   * must be in your classpath.
    +   */
    --- End diff --
    
    This also (javadoc-style)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45426985
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/IBlobWatcher.java ---
    @@ -0,0 +1,25 @@
    +/**
    + * 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;
    +
    +/**
    + * Provides a way to know when a blob changes.
    + */
    +public interface IBlobWatcher {
    +    public void blobChanged(String key);
    +}
    --- End diff --
    
    This interface may not be needed if we remove watching methods from the API.  See [comment](https://github.com/apache/storm/pull/845/files#r45425945)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45432555
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStoreAclHandler.java ---
    @@ -0,0 +1,401 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.AccessControl;
    +import backtype.storm.generated.AccessControlType;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.security.auth.AuthUtils;
    +import backtype.storm.security.auth.IPrincipalToLocal;
    +import backtype.storm.security.auth.NimbusPrincipal;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.security.Principal;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Provides common handling of acls for Blobstores.
    + * Also contains some static utility functions related to Blobstores.
    + */
    +public class BlobStoreAclHandler {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStoreAclHandler.class);
    +  private final IPrincipalToLocal _ptol;
    +
    +  public static final int READ = 0x01;
    +  public static final int WRITE = 0x02;
    +  public static final int ADMIN = 0x04;
    +  public static final List<AccessControl> WORLD_EVERYTHING =
    +      Arrays.asList(new AccessControl(AccessControlType.OTHER, READ | WRITE | ADMIN));
    +  public static final List<AccessControl> DEFAULT = new ArrayList<AccessControl>();
    +  private Set<String> _supervisors;
    +  private Set<String> _admins;
    +
    +  public BlobStoreAclHandler(Map conf) {
    +    _ptol = AuthUtils.GetPrincipalToLocalPlugin(conf);
    +    _supervisors = new HashSet<String>();
    +    _admins = new HashSet<String>();
    +    if (conf.containsKey(Config.NIMBUS_SUPERVISOR_USERS)) {
    +      _supervisors.addAll((List<String>)conf.get(Config.NIMBUS_SUPERVISOR_USERS));
    +    }
    +    if (conf.containsKey(Config.NIMBUS_ADMINS)) {
    +      _admins.addAll((List<String>)conf.get(Config.NIMBUS_ADMINS));
    +    }
    +  }
    +
    +  private static AccessControlType parseACLType(String type) {
    +    if ("other".equalsIgnoreCase(type) || "o".equalsIgnoreCase(type)) {
    +      return AccessControlType.OTHER;
    +    } else if ("user".equalsIgnoreCase(type) || "u".equalsIgnoreCase(type)) {
    +      return AccessControlType.USER;
    +    }
    +    throw new IllegalArgumentException(type+" is not a valid access control type");
    +  }
    +
    +  private static int parseAccess(String access) {
    +    int ret = 0;
    +    for (char c: access.toCharArray()) {
    +      if ('r' == c) {
    +        ret = ret | READ;
    +      } else if ('w' == c) {
    +        ret = ret | WRITE;
    +      } else if ('a' == c) {
    +        ret = ret | ADMIN;
    +      } else if ('-' == c) {
    +        //ignored
    +      } else {
    +        throw new IllegalArgumentException("");
    +      }
    +    }
    +    return ret;
    +  }
    +
    +  public static AccessControl parseAccessControl(String str) {
    +    String[] parts = str.split(":");
    +    String type = "other";
    +    String name = "";
    +    String access = "-";
    +    if (parts.length > 3) {
    +      throw new IllegalArgumentException("Don't know how to parse "+str+" into an ACL value");
    +    } else if (parts.length == 1) {
    +      type = "other";
    +      name = "";
    +      access = parts[0];
    +    } else if (parts.length == 2) {
    +      type = "user";
    +      name = parts[0];
    +      access = parts[1];
    +    } else if (parts.length == 3) {
    +      type = parts[0];
    +      name = parts[1];
    +      access = parts[2];
    +    }
    +    AccessControl ret = new AccessControl();
    +    ret.set_type(parseACLType(type));
    +    ret.set_name(name);
    +    ret.set_access(parseAccess(access));
    +    return ret;
    +  }
    +
    +  private static String accessToString(int access) {
    +    StringBuffer ret = new StringBuffer();
    +    ret.append(((access & READ) > 0) ? "r" : "-");
    +    ret.append(((access & WRITE) > 0) ? "w" : "-");
    +    ret.append(((access & ADMIN) > 0) ? "a" : "-");
    +    return ret.toString();
    +  }
    +
    +  public static String accessControlToString(AccessControl ac) {
    +    StringBuffer ret = new StringBuffer();
    +    switch(ac.get_type()) {
    +      case OTHER:
    +        ret.append("o");
    +        break;
    +      case USER:
    +        ret.append("u");
    +        break;
    +      default:
    +        throw new IllegalArgumentException("Ahh don't know what a type of "+ac.get_type()+" means ");
    +    }
    +    ret.append(":");
    +    if (ac.is_set_name()) {
    +      ret.append(ac.get_name());
    +    }
    +    ret.append(":");
    +    ret.append(accessToString(ac.get_access()));
    +    return ret.toString();
    +  }
    +
    +  public static void validateSettableACLs(String key, List<AccessControl> acls) throws AuthorizationException {
    +    Set<String> aclUsers = new HashSet<>();
    +    List<String> duplicateUsers = new ArrayList<>();
    +    for (AccessControl acl : acls) {
    +      String aclUser = acl.get_name();
    +      if (aclUser != null && !aclUser.isEmpty() && !aclUsers.add(aclUser)) {
    +        LOG.error("'{}' user can't appear more than once in the ACLs", aclUser);
    +        duplicateUsers.add(aclUser);
    +      }
    +    }
    +    if (duplicateUsers.size() > 0) {
    +      String errorMessage  = "user " + Arrays.toString(duplicateUsers.toArray())
    +          + " can't appear more than once in the ACLs for key [" + key +"].";
    +      throw new AuthorizationException(errorMessage);
    +    }
    +  }
    +
    +  private Set<String> constructUserFromPrincipals(Subject who) {
    +    Set<String> user = new HashSet<String>();
    +    if (who != null) {
    +      for (Principal p : who.getPrincipals()) {
    +        user.add(_ptol.toLocal(p));
    +      }
    +    }
    +    return user;
    +  }
    +
    +  private boolean isAdmin(Subject who) {
    +    Set<String> user = constructUserFromPrincipals(who);
    +    for (String u : user) {
    +      if (_admins.contains(u)) {
    +        return true;
    +      }
    +    }
    +    return false;
    +  }
    +
    +  private boolean isReadOperation(int operation) {
    +    if (operation == 1) {
    +      return true;
    +    }
    +    return false;
    +  }
    +
    +  private boolean isSupervisor(Subject who, int operation) {
    +    Set<String> user = constructUserFromPrincipals(who);
    +    if (isReadOperation(operation)) {
    +      for (String u : user) {
    +        if (_supervisors.contains(u)) {
    +          return true;
    +        }
    +      }
    +    }
    +    return false;
    +  }
    +
    +  private boolean isNimbus(Subject who) {
    +    Set<Principal> principals = null;
    +    boolean isNimbusInstance = false;
    +    if (who != null) {
    +      principals = who.getPrincipals();
    +      for (Principal principal : principals) {
    +        if (principal instanceof NimbusPrincipal) {
    +          isNimbusInstance = true;
    +        }
    +      }
    +    }
    +    return isNimbusInstance;
    +  }
    +
    +  public boolean checkForValidUsers(Subject who, int mask) {
    +    if (isNimbus(who) || isAdmin(who) || isSupervisor(who,mask)) {
    +      return true;
    +    }
    +    return false;
    +  }
    +
    +  /**
    +   * The user should be able to see the metadata if and only if they have any of READ, WRITE, or ADMIN
    +   */
    +  public void validateUserCanReadMeta(List<AccessControl> acl, Subject who, String key) throws AuthorizationException {
    +    hasAnyPermissions(acl, (READ|WRITE|ADMIN), who, key);
    +  }
    +
    +  /**
    +   * Validates if the user has any of the permissions
    +   * mentioned in the mask.
    +   * @param acl ACL for the key.
    +   * @param mask mask holds the cummulative value of
    +   * READ = 1, WRITE = 2 or ADMIN = 4 permissions.
    +   * mask = 1 implies READ privilege.
    +   * mask = 5 implies READ and ADMIN privileges.
    +   * @param who Is the user against whom the permissions
    +   * are validated for a key using the ACL and the mask.
    +   * @param key Key used to identify the blob.
    +   * @throws AuthorizationException
    +   */
    +  public void hasAnyPermissions(List<AccessControl> acl, int mask, Subject who, String key) throws AuthorizationException {
    +    Set<String> user = constructUserFromPrincipals(who);
    +    LOG.debug("user {}", user);
    +    if (checkForValidUsers(who, mask)) {
    +     return;
    +    }
    +    for (AccessControl ac : acl) {
    +      int allowed = getAllowed(ac, user);
    +      LOG.debug(" user: {} allowed: {} key: {}", user, allowed, key);
    +      if ((allowed & mask) > 0) {
    +        return;
    +      }
    +    }
    +    throw new AuthorizationException(
    +            user + " does not have access to " + key);
    +  }
    +
    +  /**
    +   * Validates if the user has atleast the set of permissions
    +   * mentioned in the mask.
    +   * @param acl ACL for the key.
    +   * @param mask mask holds the cummulative value of
    +   * READ = 1, WRITE = 2 or ADMIN = 4 permissions.
    +   * mask = 1 implies READ privilege.
    +   * mask = 5 implies READ and ADMIN privileges.
    +   * @param who Is the user against whom the permissions.
    +   * are validated for a key using the ACL and the mask.
    +   * @param key Key used to identify the blob.
    +   * @throws AuthorizationException
    +   */
    +  public void hasPermissions(List<AccessControl> acl, int mask, Subject who, String key) throws AuthorizationException {
    +    Set<String> user = constructUserFromPrincipals(who);
    +    LOG.debug("user {}", user);
    +    if (checkForValidUsers(who, mask)) {
    +      return;
    +    }
    +    for (AccessControl ac : acl) {
    +      int allowed = getAllowed(ac, user);
    +      mask = ~allowed & mask;
    +      LOG.debug(" user: {} allowed: {} disallowed: {} key: {}", user, allowed, mask, key);
    +    }
    +    if (mask == 0) {
    +      return;
    +    }
    +    throw new AuthorizationException(
    +            user + " does not have " + namedPerms(mask) + " access to " + key);
    +  }
    +
    +  public void normalizeSettableBlobMeta(String key, SettableBlobMeta meta, Subject who, int opMask) {
    +    meta.set_acl(normalizeSettableACLs(key, meta.get_acl(), who, opMask));
    +  }
    +
    +  private String namedPerms(int mask) {
    +    StringBuffer b = new StringBuffer();
    +    b.append("[");
    +    if ((mask & READ) > 0) {
    +      b.append("READ ");
    +    }
    +    if ((mask & WRITE) > 0) {
    +      b.append("WRITE ");
    +    }
    +    if ((mask & ADMIN) > 0) {
    +      b.append("ADMIN ");
    +    }
    +    b.append("]");
    +    return b.toString();
    +  }
    +
    +  private int getAllowed(AccessControl ac, Set<String> users) {
    +    switch (ac.get_type()) {
    +      case OTHER:
    +        return ac.get_access();
    +      case USER:
    +        if (users.contains(ac.get_name())) {
    +          return ac.get_access();
    +        }
    +        return 0;
    +      default:
    +        return 0;
    +    }
    +  }
    +
    +  private List<AccessControl> removeBadACLs(List<AccessControl> accessControls) {
    +    List<AccessControl> resultAcl = new ArrayList<AccessControl>();
    +    for (AccessControl control : accessControls) {
    +      if(control.get_type().equals(AccessControlType.OTHER) && (control.get_access() == 0 )) {
    +        LOG.debug("Removing invalid blobstore world ACL " +
    +            BlobStoreAclHandler.accessControlToString(control));
    +        continue;
    +      }
    +      resultAcl.add(control);
    +    }
    +    return resultAcl;
    +  }
    +
    +  private final List<AccessControl> normalizeSettableACLs(String key, List<AccessControl> acls, Subject who,
    +                                                    int opMask) {
    +    List<AccessControl> cleanAcls = removeBadACLs(acls);
    +    Set<String> userNames = getUserNamesFromSubject(who);
    +    for (String user : userNames) {
    +      fixACLsForUser(cleanAcls, user, opMask);
    +    }
    +    if ((who == null || userNames.isEmpty()) && !worldEverything(acls)) {
    +        cleanAcls.addAll(BlobStoreAclHandler.WORLD_EVERYTHING);
    +      LOG.debug("Access Control for key {} is normalized to world everything {}", key, cleanAcls);
    +      if (!acls.isEmpty())
    +        LOG.warn("Access control for blob with key {} is normalized to WORLD_EVERYTHING", key);
    +    }
    +    return cleanAcls;
    +  }
    +
    +  private boolean worldEverything(List<AccessControl> acls) {
    +    boolean isWorldEverything = false;
    +    for (AccessControl acl : acls) {
    +      if (acl.get_type() == AccessControlType.OTHER && acl.get_access() == 7) {
    --- End diff --
    
    Let's replace `7` with a mask of the corresponding permissions.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45383300
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStore.java ---
    @@ -0,0 +1,446 @@
    +/**
    + * 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 java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Set;
    +import java.util.regex.Pattern;
    +
    +import javax.security.auth.Subject;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import backtype.storm.daemon.Shutdownable;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +
    +/**
    + * Provides a way to store blobs that can be downloaded.
    + * Blobs must be able to be uploaded and listed from Nimbus,
    + * and downloaded from the Supervisors. It is a key value based
    + * store. Key being a string and value being the blob data.
    + *
    + * ACL checking must take place against the provided subject.
    + * If the blob store does not support Security it must validate
    + * that all ACLs set are always WORLD, everything.
    + *
    + * The users can upload their blobs through the blob store command
    + * line. The command line utilty also allows us to update,
    + * delete.
    + *
    + * Modifying the replication factor only works for HdfsBlobStore
    + * as for the LocalFsBlobStore the replication is dependent on
    + * the number of Nimbodes available.
    + */
    +public abstract class BlobStore implements Shutdownable {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStore.class);
    +  private static final Pattern KEY_PATTERN = Pattern.compile("^[\\w \\t\\.:_-]+$");
    +  protected static final String BASE_BLOBS_DIR_NAME = "blobs";
    +
    +  /**
    +   * Allows us to initialize the blob store
    +   * @param conf The storm configuration
    +   * @param baseDir The directory path to store the blobs
    +   * @param nimbusInfo Contains the nimbus host, port and leadership information.
    +   */
    +  public abstract void prepare(Map conf, String baseDir, NimbusInfo nimbusInfo);
    +
    +  /**
    +   * Creates the blob.
    +   * @param key Key for the blob.
    +   * @param meta Metadata which contains the acls information
    +   * @param who Is the subject creating the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyAlreadyExistsException
    +   */
    +  public abstract AtomicOutputStream createBlob(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException;
    +
    +  /**
    +   * Updates the blob data.
    +   * @param key Key for the blob.
    +   * @param who Is the subject having the write privilege for the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract AtomicOutputStream updateBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Gets the current version of metadata for a blob
    +   * to be viewed by the user or downloaded by the supervisor.
    +   * @param key Key for the blob.
    +   * @param who Is the subject having the read privilege for the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract ReadableBlobMeta getBlobMeta(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Sets the metadata with renewed acls for the blob.
    +   * @param key Key for the blob.
    +   * @param meta Metadata which contains the updated
    +   * acls information.
    +   * @param who Is the subject having the write privilege for the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract void setBlobMeta(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Deletes the blob data and metadata.
    +   * @param key Key for the blob.
    +   * @param who Is the subject having write privilege for the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract void deleteBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Gets the InputStream to read the blob details
    +   * @param key Key for the blob.
    +   * @param who Is the subject having the read privilege for the blob.
    +   * @return InputStreamWithMeta has the additional
    +   * file length and version information.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract InputStreamWithMeta getBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Returns an iterator with all the list of
    +   * keys currently available on the blob store.
    +   * @param who Is the subject creating the blob.
    +   * @return Iterator<String>
    +   */
    +  public abstract Iterator<String> listKeys(Subject who);
    +
    +  /**
    +   * Gets the replication factor of the blob.
    +   * @param key Key for the blob.
    +   * @param who Is the subject having the read privilege for the blob.
    +   * @return BlobReplication object containing the
    +   * replication factor for the blob.
    +   * @throws Exception
    +   */
    +  public abstract int getBlobReplication(String key, Subject who) throws Exception;
    +
    +  /**
    +   * Modifies the replication factor of the blob.
    +   * @param key Key for the blob.
    +   * @param replication The replication factor the
    +   * blob has to be set to.
    +   * @param who Is the subject having the update privilege for the blob
    +   * @return BlobReplication object containing the
    +   * updated replication factor for the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   * @throws IOException
    +   */
    +  public abstract int updateBlobReplication(String key, int replication, Subject who) throws AuthorizationException, KeyNotFoundException, IOException;
    +
    +  /**
    +   * Filters keys based on the KeyFilter
    +   * passed as the argument.
    +   * @param filter Filter passed as the
    +   * @param who Might not want to have the subject as it is not doing anything
    --- End diff --
    
    Need to remove `who` from this signature too.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45760346
  
    --- Diff: external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java ---
    @@ -0,0 +1,383 @@
    +/**
    + * 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 org.apache.storm.hdfs.blobstore;
    +
    +import backtype.storm.Config;
    +import backtype.storm.blobstore.AtomicOutputStream;
    +import backtype.storm.blobstore.AtomicOutputStream;
    --- End diff --
    
    @d2r addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43939576
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -346,53 +400,104 @@
           [(.getNodeId slot) (.getPort slot)]
           )))
     
    +(defn- get-metadata-version [blob-store key subject]
    +  (let [blob-meta (.getBlobMeta blob-store key subject)]
    +    (.get_version blob-meta)))
    +
    +(defn get-key-set-from-blob-store [blob-store]
    +  (let [key-iter (.listKeys blob-store get-nimbus-subject)]
    +    (set (iterator-seq key-iter))))
    +
     (defn- setup-storm-code [nimbus conf storm-id tmp-jar-location storm-conf topology]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -   (log-message "nimbus file location:" stormroot)
    -   (FileUtils/forceMkdir (File. stormroot))
    -   (FileUtils/cleanDirectory (File. stormroot))
    -   (setup-jar conf tmp-jar-location stormroot)
    -   (FileUtils/writeByteArrayToFile (File. (master-stormcode-path stormroot)) (Utils/serialize topology))
    -   (FileUtils/writeByteArrayToFile (File. (master-stormconf-path stormroot)) (Utils/toCompressedJsonConf storm-conf))
    -   (if (:code-distributor nimbus) (.upload (:code-distributor nimbus) stormroot storm-id))
    -   ))
    +  (let [subject (get-subject)
    +        storm-cluster-state (:storm-cluster-state nimbus)
    +        blob-store (:blob-store nimbus)
    +        jar-key (master-stormjar-key storm-id)
    +        code-key (master-stormcode-key storm-id)
    +        conf-key (master-stormconf-key storm-id)
    +        nimbus-host-port-info (:nimbus-host-port-info nimbus)]
    +    (if tmp-jar-location ;;in local mode there is no jar
    +      (do
    +        (.createBlob blob-store jar-key (FileInputStream. tmp-jar-location) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +        (if (instance? LocalFsBlobStore blob-store)
    +          (.setup-blobstore! storm-cluster-state jar-key nimbus-host-port-info (get-metadata-version blob-store jar-key subject)))))
    +    (.createBlob blob-store conf-key (Utils/toCompressedJsonConf storm-conf) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state conf-key nimbus-host-port-info (get-metadata-version blob-store conf-key subject)))
    +    (.createBlob blob-store code-key (Utils/serialize topology) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state code-key nimbus-host-port-info (get-metadata-version blob-store code-key subject)))))
    +
    +(defn- read-storm-topology [storm-id blob-store]
    +  (Utils/deserialize
    +    (.readBlob blob-store (master-stormcode-key storm-id) (get-subject)) StormTopology))
    +
    +(defn- get-blob-replication-count [blob-key nimbus]
    +  (if (:blob-store nimbus)
    +          (-> (:blob-store nimbus)
    +            (.getBlobReplication  blob-key get-nimbus-subject)
    +            (.get_replication))))
     
     (defn- wait-for-desired-code-replication [nimbus conf storm-id]
       (let [min-replication-count (conf TOPOLOGY-MIN-REPLICATION-COUNT)
             max-replication-wait-time (conf TOPOLOGY-MAX-REPLICATION-WAIT-TIME-SEC)
    -        total-wait-time (atom 0)
    -        current-replication-count (atom (if (:code-distributor nimbus) (.getReplicationCount (:code-distributor nimbus) storm-id) 0))]
    -  (if (:code-distributor nimbus)
    -    (while (and (> min-replication-count @current-replication-count)
    -             (or (= -1 max-replication-wait-time)
    -               (< @total-wait-time max-replication-wait-time)))
    +        current-replication-count-jar (if (not (local-mode? conf)) (atom
    +                                                                     (get-blob-replication-count (master-stormjar-key storm-id) nimbus))
    +                                                                     (atom min-replication-count))
    +        current-replication-count-code (atom (get-blob-replication-count (master-stormcode-key storm-id) nimbus))
    +        current-replication-count-conf (atom (get-blob-replication-count (master-stormconf-key storm-id) nimbus))
    +        total-wait-time (atom 0)]
    +    (log-message "wait for desired replication" "count"
    +      min-replication-count "wait-time" max-replication-wait-time
    +      "code" @current-replication-count-code
    +      "conf" @current-replication-count-conf
    +      "jar" @current-replication-count-jar
    +      "replication count" (get-blob-replication-count (master-stormconf-key storm-id) nimbus))
    --- End diff --
    
    minor: could we add spaces to make the log message more readable?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r44997265
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -671,6 +787,11 @@
                             0
                             (conf SUPERVISOR-HEARTBEAT-FREQUENCY-SECS)
                             heartbeat-fn)
    +    (doseq [storm-id downloaded-storm-ids] (add-blob-references (:localizer supervisor) storm-id
    +                                             conf))
    --- End diff --
    
    doseq body should have its own line. Then conf can be moved back into the same line as the function it is being passed to.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on the pull request:

    https://github.com/apache/storm/pull/845#issuecomment-158261254
  
    Utils.java:
    
    ```
    1252     // Normalize state
    1253     public static String[] normalizeVersionInfo(String nimbusKeyVersionInfo) {
    1254       String[] normalizeVersionInfo = new String[2];
    1255       int lastIndex = nimbusKeyVersionInfo.lastIndexOf("-");
    1256       normalizeVersionInfo[0] = nimbusKeyVersionInfo.substring(0, lastIndex);
    1257       normalizeVersionInfo[1] = nimbusKeyVersionInfo.substring(lastIndex + 1);
    1258       return normalizeVersionInfo;
    1259     }
    ```
    
    We should make a class out of BlobVersion, with `name` and `version` as members. It's easier to understand than a `String[]` value.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43814165
  
    --- Diff: storm-core/src/clj/backtype/storm/util.clj ---
    @@ -53,6 +57,16 @@
         e
         (RuntimeException. e)))
     
    +(defn nimbus-subject []
    +  (let [subject (Subject.)
    +        principal (NimbusPrincipal.)
    +        principals (.getPrincipals subject)]
    +    (.add principals principal)
    +    subject))
    --- End diff --
    
    minor nit: we are supposed to put the argument list on its own line now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/storm/pull/845


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by unsleepy22 <gi...@git.apache.org>.
Github user unsleepy22 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46017804
  
    --- Diff: external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java ---
    @@ -0,0 +1,379 @@
    +/**
    + * 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 org.apache.storm.hdfs.blobstore;
    +
    +import backtype.storm.Config;
    +import backtype.storm.blobstore.AtomicOutputStream;
    +import backtype.storm.blobstore.BlobStore;
    +import backtype.storm.blobstore.BlobStoreAclHandler;
    +import backtype.storm.blobstore.BlobStoreFile;
    +import backtype.storm.blobstore.InputStreamWithMeta;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.utils.Utils;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.security.UserGroupInformation;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.ByteArrayOutputStream;
    +import java.io.FileNotFoundException;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.security.AccessController;
    +import java.security.PrivilegedAction;
    +import java.util.Iterator;
    +import java.util.Map;
    +
    +import static backtype.storm.blobstore.BlobStoreAclHandler.ADMIN;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.READ;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.WRITE;
    +
    +/**
    + * Provides a HDFS file system backed blob store implementation.
    + * Note that this provides an api for having HDFS be the backing store for the blobstore,
    + * it is not a service/daemon.
    + */
    +public class HdfsBlobStore extends BlobStore {
    +  public static final Logger LOG = LoggerFactory.getLogger(HdfsBlobStore.class);
    +  private static final String DATA_PREFIX = "data_";
    +  private static final String META_PREFIX = "meta_";
    +  private BlobStoreAclHandler _aclHandler;
    +  private HdfsBlobStoreImpl _hbs;
    +  private Subject _localSubject;
    +  private Map conf;
    +
    +  /**
    +   * Get the subject from Hadoop so we can use it to validate the acls. There is no direct
    +   * interface from UserGroupInformation to get the subject, so do a doAs and get the context.
    +   * We could probably run everything in the doAs but for now just grab the subject.
    +   */
    +  private Subject getHadoopUser() {
    +    Subject subj;
    +    try {
    +      subj = UserGroupInformation.getCurrentUser().doAs(
    +          new PrivilegedAction<Subject>() {
    +            @Override
    +            public Subject run() {
    +              return Subject.getSubject(AccessController.getContext());
    +            }
    +          });
    +    } catch (IOException e) {
    +      throw new RuntimeException("Error creating subject and logging user in!", e);
    +    }
    +    return subj;
    +  }
    +
    +  /**
    +   * If who is null then we want to use the user hadoop says we are.
    +   * Required for the supervisor to call these routines as its not
    +   * logged in as anyone.
    +   */
    +  private Subject checkAndGetSubject(Subject who) {
    +    if (who == null) {
    +      return _localSubject;
    +    }
    +    return who;
    +  }
    +
    +  @Override
    +  public void prepare(Map conf, String overrideBase, NimbusInfo nimbusInfo) {
    +    this.conf = conf;
    +    prepareInternal(conf, overrideBase, null);
    +  }
    +
    +  /**
    +   * Allow a Hadoop Configuration to be passed for testing. If it's null then the hadoop configs
    +   * must be in your classpath.
    +   */
    +  protected void prepareInternal(Map conf, String overrideBase, Configuration hadoopConf) {
    +    this.conf = conf;
    +    if (overrideBase == null) {
    +      overrideBase = (String)conf.get(Config.BLOBSTORE_DIR);
    +    }
    +    if (overrideBase == null) {
    +      throw new RuntimeException("You must specify a blobstore directory for HDFS to use!");
    +    }
    +    LOG.debug("directory is: {}", overrideBase);
    +    try {
    +      // if a HDFS keytab/principal have been supplied login, otherwise assume they are
    +      // logged in already or running insecure HDFS.
    +      if (conf.get(Config.BLOBSTORE_HDFS_PRINCIPAL) != null &&
    --- End diff --
    
    could you extract principal & keytab local params as they're called 3 times


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by unsleepy22 <gi...@git.apache.org>.
Github user unsleepy22 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46024038
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStore.java ---
    @@ -0,0 +1,296 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.utils.Utils;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.ByteArrayOutputStream;
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.FileNotFoundException;
    +import java.io.InputStream;
    +
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;;
    +
    +import static backtype.storm.blobstore.BlobStoreAclHandler.ADMIN;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.READ;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.WRITE;
    +
    +/**
    + * Provides a local file system backed blob store implementation for Nimbus.
    + */
    +public class LocalFsBlobStore extends BlobStore {
    +  public static final Logger LOG = LoggerFactory.getLogger(LocalFsBlobStore.class);
    +  private static final String DATA_PREFIX = "data_";
    +  private static final String META_PREFIX = "meta_";
    +  protected BlobStoreAclHandler _aclHandler;
    +  private final String BLOBSTORE_SUBTREE = "/blobstore/";
    +  private NimbusInfo nimbusInfo;
    +  private FileBlobStoreImpl fbs;
    +  private Map conf;
    +
    +  @Override
    +  public void prepare(Map conf, String overrideBase, NimbusInfo nimbusInfo) {
    +    this.conf = conf;
    +    this.nimbusInfo = nimbusInfo;
    +     if (overrideBase == null) {
    +       overrideBase = (String)conf.get(Config.BLOBSTORE_DIR);
    +      if (overrideBase == null) {
    +        overrideBase = (String) conf.get(Config.STORM_LOCAL_DIR);
    +      }
    +    }
    +    File baseDir = new File(overrideBase, BASE_BLOBS_DIR_NAME);
    +    try {
    +      fbs = new FileBlobStoreImpl(baseDir, conf);
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +    _aclHandler = new BlobStoreAclHandler(conf);
    +  }
    +
    +  @Override
    +  public AtomicOutputStream createBlob(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException {
    +    LOG.debug("Creating Blob for key {}", key);
    +    validateKey(key);
    +    _aclHandler.normalizeSettableBlobMeta(key, meta, who, READ | WRITE | ADMIN);
    --- End diff --
    
    I've seen READ | WRITE | ADMIN occur a few times, should we define a constant for this?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by hustfxj <gi...@git.apache.org>.
Github user hustfxj commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45290503
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -732,25 +854,90 @@
      (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
       (worker-launcher-and-wait conf (storm-conf TOPOLOGY-SUBMITTER-USER) ["code-dir" dir] :log-prefix (str "setup conf for " dir))))
     
    +(defn setup-blob-permission [conf storm-conf path]
    +  (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
    +    (worker-launcher-and-wait conf (storm-conf TOPOLOGY-SUBMITTER-USER) ["blob" path] :log-prefix (str "setup blob permissions for " path))))
    +
    +(defn setup-storm-code-dir [conf storm-conf dir]
    +  (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
    +    (worker-launcher-and-wait conf (storm-conf TOPOLOGY-SUBMITTER-USER) ["code-dir" dir] :log-prefix (str "setup conf for " dir))))
    +
    +(defn download-blobs-for-topology!
    +  "Download all blobs listed in the topology configuration for a given topology."
    +  [conf stormconf-path localizer tmproot]
    +  (let [storm-conf (read-supervisor-storm-conf-given-path conf stormconf-path)
    +        blobstore-map (storm-conf TOPOLOGY-BLOBSTORE-MAP)
    +        user (storm-conf TOPOLOGY-SUBMITTER-USER)
    +        topo-name (storm-conf TOPOLOGY-NAME)
    +        user-dir (.getLocalUserFileCacheDir localizer user)
    +        localresources (blobstore-map-to-localresources blobstore-map)]
    +    (when localresources
    +      (when-not (.exists user-dir)
    +        (FileUtils/forceMkdir user-dir)
    +        (setup-blob-permission conf storm-conf (.toString user-dir)))
    +      (try
    +        (let [localized-resources (.getBlobs localizer localresources user topo-name user-dir)]
    +          (setup-blob-permission conf storm-conf (.toString user-dir))
    +          (doseq [local-rsrc localized-resources]
    +            (let [rsrc-file-path (File. (.getFilePath local-rsrc))
    +                  key-name (.getName rsrc-file-path)
    +                  blob-symlink-target-name (.getName (File. (.getCurrentSymlinkPath local-rsrc)))
    +                  symlink-name (get-blob-localname (get blobstore-map key-name) key-name)]
    +              (create-symlink! tmproot (.getParent rsrc-file-path) symlink-name
    +                blob-symlink-target-name))))
    +        (catch AuthorizationException authExp
    +          (log-error authExp))
    +        (catch KeyNotFoundException knf
    +          (log-error knf))))))
    +
    +(defn get-blob-file-names
    +  [blobstore-map]
    +  (if blobstore-map
    +    (for [[k, data] blobstore-map]
    +      (get-blob-localname data k))))
    +
    +(defn download-blobs-for-topology-succeed?
    +  "Assert if all blobs are downloaded for the given topology"
    +  [stormconf-path target-dir]
    +  (let [storm-conf (clojurify-structure (Utils/fromCompressedJsonConf (FileUtils/readFileToByteArray (File. stormconf-path))))
    +        blobstore-map (storm-conf TOPOLOGY-BLOBSTORE-MAP)
    +        file-names (get-blob-file-names blobstore-map)]
    +    (if (and file-names (> (count file-names) 0))
    +      (every? #(Utils/checkFileExists target-dir %) file-names)
    +      true)))
    +
     ;; distributed implementation
     (defmethod download-storm-code
    -    :distributed [conf storm-id master-code-dir supervisor download-lock]
    -    ;; Downloading to permanent location is atomic
    -    (let [tmproot (str (supervisor-tmp-dir conf) file-path-separator (uuid))
    -          stormroot (supervisor-stormdist-root conf storm-id)
    -          master-meta-file-path (master-storm-metafile-path master-code-dir)
    -          supervisor-meta-file-path (supervisor-storm-metafile-path tmproot)]
    -      (locking download-lock
    -        (log-message "Downloading code for storm id " storm-id " from " master-code-dir)
    -        (FileUtils/forceMkdir (File. tmproot))
    -        (Utils/downloadFromMaster conf master-meta-file-path supervisor-meta-file-path)
    -        (if (:code-distributor supervisor)
    -          (.download (:code-distributor supervisor) storm-id (File. supervisor-meta-file-path)))
    -        (extract-dir-from-jar (supervisor-stormjar-path tmproot) RESOURCES-SUBDIR tmproot)
    -        (if (.exists (File. stormroot)) (FileUtils/forceDelete (File. stormroot)))
    -        (FileUtils/moveDirectory (File. tmproot) (File. stormroot))
    -        (setup-storm-code-dir conf (read-supervisor-storm-conf conf storm-id) stormroot)
    -        (log-message "Finished downloading code for storm id " storm-id " from " master-code-dir))))
    +  :distributed [conf storm-id master-code-dir localizer]
    +  ;; Downloading to permanent location is atomic
    +  (let [tmproot (str (supervisor-tmp-dir conf) file-path-separator (uuid))
    +        stormroot (supervisor-stormdist-root conf storm-id)
    +        blobstore (Utils/getSupervisorBlobStore conf)]
    --- End diff --
    
    I hava a small suggestions.  for example, this blobstore which is supervisor's client about blobstore maybe make me confused with real "Blobstore". can you change other name?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45760735
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/KeyVersion.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * 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.zookeeper.CreateMode;
    +import org.apache.zookeeper.ZooDefs;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.TreeSet;
    +import java.util.Map;
    +import java.util.List;
    +
    +/**
    + * Class hands over the version of the key to be stored within the zookeeper
    + */
    +public class KeyVersion {
    +  private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
    +  private final String BLOBSTORE_SUBTREE="/blobstore";
    +  private final String BLOBSTORE_KEY_COUNTER_SUBTREE="/blobstorekeycounter";
    +  private String key;
    +  private NimbusInfo nimbusInfo;
    +
    +  public KeyVersion(String key, NimbusInfo nimbusInfo) {
    +    this.key = key;
    +    this.nimbusInfo = nimbusInfo;
    +  }
    +
    +  public int getKeyVersion(Map conf) {
    +    TreeSet<Integer> versions = new TreeSet<Integer>();
    +    CuratorFramework zkClient = Utils.createZKClient(conf);
    +    try {
    +      // Key has not been created yet and it is the first time it is being created
    +      if(zkClient.checkExists().forPath(BLOBSTORE_SUBTREE + "/" + key) == null) {
    +        zkClient.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT)
    +                .withACL(ZooDefs.Ids.OPEN_ACL_UNSAFE).forPath(BLOBSTORE_KEY_COUNTER_SUBTREE + "/" + key + "/" + 1);
    +        return 1;
    +      }
    +
    +      // When all nimbodes go down and one or few of them come up
    +      // Unfortunately there might not be an exact way to know which one contains the most updated blob
    +      // if all go down which is unlikely. Hence there might be a need to update the blob if all go down
    +      List<String> stateInfoList = zkClient.getChildren().forPath(BLOBSTORE_SUBTREE + "/" + key);
    +      LOG.debug("stateInfoList {} stateInfoList {}", stateInfoList.size(), stateInfoList);
    --- End diff --
    
    @d2r addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43781520
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -205,6 +249,10 @@
                           (log-message "Killing topology: " storm-id)
                           (.remove-storm! (:storm-cluster-state nimbus)
                                           storm-id)
    +                      (if (instance? LocalFsBlobStore (:blob-store nimbus))
    --- End diff --
    
    We do not want to create a state for hdfs blobstore as the interaction between the daemons (nimbus and supervisor) with hdfs is direct and CRUD operations are handled by HDFS


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45760778
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/KeyVersion.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * 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.zookeeper.CreateMode;
    +import org.apache.zookeeper.ZooDefs;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.TreeSet;
    +import java.util.Map;
    +import java.util.List;
    +
    +/**
    + * Class hands over the version of the key to be stored within the zookeeper
    + */
    +public class KeyVersion {
    +  private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
    +  private final String BLOBSTORE_SUBTREE="/blobstore";
    +  private final String BLOBSTORE_KEY_COUNTER_SUBTREE="/blobstorekeycounter";
    +  private String key;
    +  private NimbusInfo nimbusInfo;
    +
    +  public KeyVersion(String key, NimbusInfo nimbusInfo) {
    +    this.key = key;
    +    this.nimbusInfo = nimbusInfo;
    +  }
    +
    +  public int getKeyVersion(Map conf) {
    +    TreeSet<Integer> versions = new TreeSet<Integer>();
    +    CuratorFramework zkClient = Utils.createZKClient(conf);
    +    try {
    +      // Key has not been created yet and it is the first time it is being created
    +      if(zkClient.checkExists().forPath(BLOBSTORE_SUBTREE + "/" + key) == null) {
    +        zkClient.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT)
    +                .withACL(ZooDefs.Ids.OPEN_ACL_UNSAFE).forPath(BLOBSTORE_KEY_COUNTER_SUBTREE + "/" + key + "/" + 1);
    +        return 1;
    +      }
    +
    +      // When all nimbodes go down and one or few of them come up
    +      // Unfortunately there might not be an exact way to know which one contains the most updated blob
    +      // if all go down which is unlikely. Hence there might be a need to update the blob if all go down
    +      List<String> stateInfoList = zkClient.getChildren().forPath(BLOBSTORE_SUBTREE + "/" + key);
    +      LOG.debug("stateInfoList {} stateInfoList {}", stateInfoList.size(), stateInfoList);
    +      if(stateInfoList.isEmpty()) {
    +        return getKeyVersionCounterValue(zkClient, key);
    +      }
    +
    +      LOG.debug("stateInfoSize {}", stateInfoList.size());
    +      // In all other cases check for the latest version on the nimbus and assign the version
    +      // check if all are have same version, if not assign the highest version
    +      for (String stateInfo:stateInfoList) {
    +        versions.add(Integer.parseInt(Utils.normalizeVersionInfo(stateInfo)[1]));
    +      }
    +
    +      int currentCounter = getKeyVersionCounterValue(zkClient, key);
    +      // This condition returns version when a nimbus crashes and comes up
    +      if (!checkIfStateContainsCurrentNimbusHost(stateInfoList, nimbusInfo) && !nimbusInfo.isLeader()) {
    +        if (versions.last() < currentCounter) {
    +          return currentCounter;
    +        } else {
    +          return currentCounter - 1;
    +        }
    +      }
    +      // Condition checks for an update scenario
    +      if (stateInfoList.size() >= 1 && versions.size() == 1) {
    +        if (versions.first() < getKeyVersionCounterValue(zkClient, key)) {
    +          incrementCounter(zkClient, key, currentCounter);
    +          return currentCounter + 1;
    +        } else {
    +          incrementCounter(zkClient, key, currentCounter);
    +          return versions.first() + 1;
    +        }
    +      }
    +    } catch(Exception e) {
    +      LOG.error("Exception {}", e);
    +    } finally {
    +      if (zkClient != null) {
    +        zkClient.close();
    +      }
    +    }
    +    return versions.last();
    +  }
    +
    +  public boolean checkIfStateContainsCurrentNimbusHost(List<String> stateInfoList, NimbusInfo nimbusInfo) {
    +    boolean containsNimbusHost = false;
    +    for(String stateInfo:stateInfoList) {
    +      if(stateInfo.contains(nimbusInfo.getHost())) {
    +        containsNimbusHost = true;
    +        break;
    +      }
    +    }
    +    return containsNimbusHost;
    +  }
    +
    +  public void incrementCounter(CuratorFramework zkClient, String key, int count) throws Exception {
    --- End diff --
    
    @d2r addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45264867
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -927,31 +1133,32 @@
            first ))
     
     (defmethod download-storm-code
    -    :local [conf storm-id master-code-dir supervisor download-lock]
    -    (let [stormroot (supervisor-stormdist-root conf storm-id)]
    -      (locking download-lock
    -            (FileUtils/copyDirectory (File. master-code-dir) (File. stormroot))
    -            (let [classloader (.getContextClassLoader (Thread/currentThread))
    -                  resources-jar (resources-jar)
    -                  url (.getResource classloader RESOURCES-SUBDIR)
    -                  target-dir (str stormroot file-path-separator RESOURCES-SUBDIR)]
    -              (cond
    -               resources-jar
    -               (do
    -                 (log-message "Extracting resources from jar at " resources-jar " to " target-dir)
    -                 (extract-dir-from-jar resources-jar RESOURCES-SUBDIR stormroot))
    -               url
    -               (do
    -                 (log-message "Copying resources at " (URI. (str url)) " to " target-dir)
    -                 (if (= (.getProtocol url) "jar" )
    -                   (extract-dir-from-jar (.getFile (.getJarFileURL (.openConnection url))) RESOURCES-SUBDIR stormroot)
    -                   (FileUtils/copyDirectory (File. (.getPath (URI. (str url)))) (File. target-dir)))
    -                 )
    -               )
    -              )
    -            )))
    -
    -(defmethod mk-code-distributor :local [conf] nil)
    +  :local [conf storm-id master-code-dir localizer]
    +  (let [tmproot (str (supervisor-tmp-dir conf) file-path-separator (uuid))
    +        stormroot (supervisor-stormdist-root conf storm-id)
    +        blob-store (Utils/getNimbusBlobStore conf master-code-dir nil)]
    +    (try
    +      (FileUtils/forceMkdir (File. tmproot))
    +      (.readBlobTo blob-store (master-stormcode-key storm-id) (FileOutputStream. (supervisor-stormcode-path tmproot)) nil)
    +      (.readBlobTo blob-store (master-stormconf-key storm-id) (FileOutputStream. (supervisor-stormconf-path tmproot)) nil)
    +      (finally
    +        (.shutdown blob-store)))
    +    (FileUtils/moveDirectory (File. tmproot) (File. stormroot))
    +    (setup-storm-code-dir conf (read-supervisor-storm-conf conf storm-id) stormroot)
    +    (let [classloader (.getContextClassLoader (Thread/currentThread))
    +          resources-jar (resources-jar)
    +          url (.getResource classloader RESOURCES-SUBDIR)
    +          target-dir (str stormroot file-path-separator RESOURCES-SUBDIR)]
    +      (cond
    +        resources-jar
    --- End diff --
    
    param `localizer` is not used


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45398547
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStoreAclHandler.java ---
    @@ -0,0 +1,387 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.AccessControl;
    +import backtype.storm.generated.AccessControlType;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.security.auth.AuthUtils;
    +import backtype.storm.security.auth.IPrincipalToLocal;
    +import backtype.storm.security.auth.NimbusPrincipal;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.security.Principal;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Provides common handling of acls for Blobstores.
    + * Also contains some static utility functions related to Blobstores.
    + */
    +public class BlobStoreAclHandler {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStoreAclHandler.class);
    +  private final IPrincipalToLocal _ptol;
    +
    +  public static final int READ = 0x01;
    +  public static final int WRITE = 0x02;
    +  public static final int ADMIN = 0x04;
    +  public static final List<AccessControl> WORLD_EVERYTHING =
    +      Arrays.asList(new AccessControl(AccessControlType.OTHER, READ | WRITE | ADMIN));
    +  public static final List<AccessControl> DEFAULT = new ArrayList<AccessControl>();
    +  private Set<String> _supervisors;
    +  private Set<String> _admins;
    +
    +  public BlobStoreAclHandler(Map conf) {
    +    _ptol = AuthUtils.GetPrincipalToLocalPlugin(conf);
    +    _supervisors = new HashSet<String>();
    +    _admins = new HashSet<String>();
    +    if (conf.containsKey(Config.NIMBUS_SUPERVISOR_USERS)) {
    +      _supervisors.addAll((List<String>)conf.get(Config.NIMBUS_SUPERVISOR_USERS));
    +    }
    +    if (conf.containsKey(Config.NIMBUS_ADMINS)) {
    +      _admins.addAll((List<String>)conf.get(Config.NIMBUS_ADMINS));
    +    }
    +  }
    +
    +  private static AccessControlType parseACLType(String type) {
    +    if ("other".equalsIgnoreCase(type) || "o".equalsIgnoreCase(type)) {
    +      return AccessControlType.OTHER;
    +    } else if ("user".equalsIgnoreCase(type) || "u".equalsIgnoreCase(type)) {
    +      return AccessControlType.USER;
    +    }
    +    throw new IllegalArgumentException(type+" is not a valid access control type");
    +  }
    +
    +  private static int parseAccess(String access) {
    +    int ret = 0;
    +    for (char c: access.toCharArray()) {
    +      if ('r' == c) {
    +        ret = ret | READ;
    +      } else if ('w' == c) {
    +        ret = ret | WRITE;
    +      } else if ('a' == c) {
    +        ret = ret | ADMIN;
    +      } else if ('-' == c) {
    +        //ignored
    +      } else {
    +        throw new IllegalArgumentException("");
    +      }
    +    }
    +    return ret;
    +  }
    +
    +  public static AccessControl parseAccessControl(String str) {
    +    String[] parts = str.split(":");
    +    String type = "other";
    +    String name = "";
    +    String access = "-";
    +    if (parts.length > 3) {
    +      throw new IllegalArgumentException("Don't know how to parse "+str+" into an ACL value");
    +    } else if (parts.length == 1) {
    +      type = "other";
    +      name = "";
    +      access = parts[0];
    +    } else if (parts.length == 2) {
    +      type = "user";
    +      name = parts[0];
    +      access = parts[1];
    +    } else if (parts.length == 3) {
    +      type = parts[0];
    +      name = parts[1];
    +      access = parts[2];
    +    }
    +    AccessControl ret = new AccessControl();
    +    ret.set_type(parseACLType(type));
    +    ret.set_name(name);
    +    ret.set_access(parseAccess(access));
    +    return ret;
    +  }
    +
    +  private static String accessToString(int access) {
    +    StringBuffer ret = new StringBuffer();
    +    ret.append(((access & READ) > 0) ? "r" : "-");
    +    ret.append(((access & WRITE) > 0) ? "w" : "-");
    +    ret.append(((access & ADMIN) > 0) ? "a" : "-");
    +    return ret.toString();
    +  }
    +
    +  public static String accessControlToString(AccessControl ac) {
    +    StringBuffer ret = new StringBuffer();
    +    switch(ac.get_type()) {
    +      case OTHER:
    +        ret.append("o");
    +        break;
    +      case USER:
    +        ret.append("u");
    +        break;
    +      default:
    +        throw new IllegalArgumentException("Ahh don't know what a type of "+ac.get_type()+" means ");
    +    }
    +    ret.append(":");
    +    if (ac.is_set_name()) {
    +      ret.append(ac.get_name());
    +    }
    +    ret.append(":");
    +    ret.append(accessToString(ac.get_access()));
    +    return ret.toString();
    +  }
    +
    +  public static void validateSettableACLs(String key, List<AccessControl> acls) throws AuthorizationException {
    +    Set<String> aclUsers = new HashSet<>();
    +    List<String> duplicateUsers = new ArrayList<>();
    +    for (AccessControl acl : acls) {
    +      String aclUser = acl.get_name();
    +      if (aclUser != null && !aclUser.isEmpty() && !aclUsers.add(aclUser)) {
    +        LOG.error("'{}' user can't appear more than once in the ACLs", aclUser);
    +        duplicateUsers.add(aclUser);
    +      }
    +    }
    +    if (duplicateUsers.size() > 0) {
    +      String errorMessage  = "user " + Arrays.toString(duplicateUsers.toArray())
    +          + " can't appear more than once in the ACLs for key [" + key +"].";
    +      throw new AuthorizationException(errorMessage);
    +    }
    +  }
    +
    +  private Set<String> constructUserFromPrincipals(Subject who) {
    +    Set<String> user = new HashSet<String>();
    +    if (who == null) {
    +      LOG.debug("in validate acl who is null");
    +    } else {
    +      LOG.debug("in validate acl: " + who);
    +    }
    +    if (who != null) {
    +      for (Principal p : who.getPrincipals()) {
    +        user.add(_ptol.toLocal(p));
    +      }
    +    }
    +    return user;
    +  }
    +
    +  private boolean isSupervisorOrAdmin(Set<String> user, int mask) {
    +    boolean isSupervisor = false;
    +    boolean isAdmin = false;
    +    for(String u : user) {
    +      if (_supervisors.contains(u)) {
    +        isSupervisor = true;
    +        break;
    +      }
    +      if (_admins.contains(u)) {
    +        isAdmin = true;
    +        break;
    +      }
    +    }
    +    if (mask > 0 && !isAdmin) {
    +      isSupervisor = (isSupervisor && (mask == 1));
    +    }
    +    return isSupervisor || isAdmin;
    +  }
    +
    +  private boolean isNimbus(Subject who) {
    +    Set<Principal> principals = null;
    +    boolean isNimbusInstance = false;
    +    if(who != null) {
    +      principals = who.getPrincipals();
    +      for (Principal principal : principals) {
    +        if (principal instanceof NimbusPrincipal) {
    +          isNimbusInstance = true;
    +        }
    +      }
    +    }
    +    return isNimbusInstance;
    +  }
    +
    +  /**
    +   * The user should be able to see the metadata if and only if they have any of READ, WRITE, or ADMIN
    +   */
    +  public void validateUserCanReadMeta(List<AccessControl> acl, Subject who, String key)
    +      throws AuthorizationException {
    +    Set<String> user = constructUserFromPrincipals(who);
    +    if (isNimbus(who)) {
    +      return;
    +    }
    +    if (isSupervisorOrAdmin(user, -1)) {
    +      return;
    +    }
    +    for (AccessControl ac : acl) {
    +      int allowed = getAllowed(ac, user);
    +      LOG.debug(" user: {} allowed: {} key: {}", user, allowed, key);
    +      if ((allowed & (READ | WRITE | ADMIN)) > 0) {
    +        return;
    +      }
    +    }
    +    throw new AuthorizationException(
    +            user + " does not have access to " + key);
    +  }
    +
    +  public void validateAnyACL(List<AccessControl> acl, int validPermissions, Subject who, String key) throws AuthorizationException {
    +    Set<String> user = constructUserFromPrincipals(who);
    +    LOG.debug("user {}", user);
    +    if (isNimbus(who)) {
    +      return;
    +    }
    +    if(isSupervisorOrAdmin(user, validPermissions)) {
    +      return;
    +    }
    +    for (AccessControl ac : acl) {
    +      int allowed = getAllowed(ac, user);
    +      LOG.debug(" user: {} allowed: {} key: {}", user, allowed, key);
    +      if ((allowed & validPermissions) > 0) {
    +        return;
    +      }
    +    }
    +    throw new AuthorizationException(
    +            user + " does not have access to " + key);
    +  }
    +
    +  //Here all acls must match
    +  public void validateACL(List<AccessControl> acl, int mask, Subject who, String key)
    +      throws AuthorizationException {
    +    Set<String> user = constructUserFromPrincipals(who);
    +    LOG.debug("user {}", user);
    +    if (isNimbus(who)) {
    +      return;
    +    }
    +    if(isSupervisorOrAdmin(user, mask)) {
    +      return;
    +    }
    +    for (AccessControl ac : acl) {
    +      int allowed = getAllowed(ac, user);
    +      mask = ~allowed & mask;
    +      LOG.debug(" user: {} allowed: {} disallowed: {} key: {}", user, allowed, mask, key);
    +    }
    +    if (mask == 0) {
    +      return;
    +    }
    +    throw new AuthorizationException(
    +            user + " does not have " + namedPerms(mask) + " access to " + key);
    +  }
    --- End diff --
    
    `validateACL` is a misnomer.  We are really validating that the subject `who` can perform a given operation `mask` on an object `key`.  First we check if the subject includes nimbus, then we check if the subject includes an admin, then we check if the subject includes the supervisor AND the operation either specifically requires only READ permission or else does not require any permission.
    
    If all of that did not permit the operation, only then do we start looking at the ACL to see if any user in the subject has at least enough permission to perform the operation on the object.  The other checks do not involve the blob's ACL at all.
    
    We could rename this to `permitOperation` or simply `permit`, and pull out the common code so there is less that can go wrong maintaining it in the future.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45761222
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStore.java ---
    @@ -0,0 +1,306 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.utils.Utils;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.thrift.TBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.ByteArrayOutputStream;
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.FileNotFoundException;
    +import java.io.InputStream;
    +
    +import java.util.*;
    --- End diff --
    
    @d2r done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43785630
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -346,53 +394,124 @@
           [(.getNodeId slot) (.getPort slot)]
           )))
     
    +(defn- get-nimbus-subject []
    +  (let [nimbus-subject (Subject.)
    +        nimbus-principal (NimbusPrincipal.)
    +        principals (.getPrincipals nimbus-subject)]
    +    (.add principals nimbus-principal)
    +    nimbus-subject))
    +
    +(defn- get-metadata-version [blob-store key subject]
    +  (let [blob-meta (.getBlobMeta blob-store key subject)]
    +    (.get_version blob-meta)))
    +
    +(defn get-key-list-from-blob-store [blob-store]
    +  (let [key-iter (.listKeys blob-store (get-nimbus-subject))
    +        keys (iterator-seq key-iter)]
    +    (if (not-nil? keys)
    +      (java.util.ArrayList. keys)
    +      [])))
    +
     (defn- setup-storm-code [nimbus conf storm-id tmp-jar-location storm-conf topology]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -   (log-message "nimbus file location:" stormroot)
    -   (FileUtils/forceMkdir (File. stormroot))
    -   (FileUtils/cleanDirectory (File. stormroot))
    -   (setup-jar conf tmp-jar-location stormroot)
    -   (FileUtils/writeByteArrayToFile (File. (master-stormcode-path stormroot)) (Utils/serialize topology))
    -   (FileUtils/writeByteArrayToFile (File. (master-stormconf-path stormroot)) (Utils/toCompressedJsonConf storm-conf))
    -   (if (:code-distributor nimbus) (.upload (:code-distributor nimbus) stormroot storm-id))
    -   ))
    +  (let [subject (get-subject)
    +        storm-cluster-state (:storm-cluster-state nimbus)
    +        blob-store (:blob-store nimbus)
    +        jar-key (master-stormjar-key storm-id)
    +        code-key (master-stormcode-key storm-id)
    +        conf-key (master-stormconf-key storm-id)
    +        nimbus-host-port-info (:nimbus-host-port-info nimbus)]
    +    (log-message "subject-changed" subject)
    +    (if tmp-jar-location ;;in local mode there is no jar
    +      (do
    +        (log-message "tmp-jar-location" tmp-jar-location)
    +        (.createBlob blob-store jar-key (FileInputStream. tmp-jar-location) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +        (if (instance? LocalFsBlobStore blob-store)
    +          (.setup-blobstore! storm-cluster-state jar-key nimbus-host-port-info (get-metadata-version blob-store jar-key subject)))
    +        ))
    +    (.createBlob blob-store conf-key (Utils/toCompressedJsonConf storm-conf) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state conf-key nimbus-host-port-info (get-metadata-version blob-store conf-key subject)))
    +    (.createBlob blob-store code-key (Utils/serialize topology) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state code-key nimbus-host-port-info (get-metadata-version blob-store code-key subject)))))
    +
    +(defn- read-storm-topology [storm-id blob-store]
    +  (Utils/deserialize
    +    (.readBlob blob-store (master-stormcode-key storm-id) (get-subject)) StormTopology))
    +
    +(defn- get-blob-replication-count [blob-key nimbus]
    +  (if (:blob-store nimbus)
    +          (-> (:blob-store nimbus)
    +            (.getBlobReplication  blob-key (get-nimbus-subject))
    +            (.get_replication))))
     
     (defn- wait-for-desired-code-replication [nimbus conf storm-id]
       (let [min-replication-count (conf TOPOLOGY-MIN-REPLICATION-COUNT)
             max-replication-wait-time (conf TOPOLOGY-MAX-REPLICATION-WAIT-TIME-SEC)
    -        total-wait-time (atom 0)
    -        current-replication-count (atom (if (:code-distributor nimbus) (.getReplicationCount (:code-distributor nimbus) storm-id) 0))]
    -  (if (:code-distributor nimbus)
    -    (while (and (> min-replication-count @current-replication-count)
    +        current-replication-count-jar (if (not (local-mode? conf)) (atom
    +                                                                     (get-blob-replication-count (master-stormjar-key storm-id) nimbus))
    +                                                                     (atom min-replication-count))
    +        current-replication-count-code (atom (get-blob-replication-count (master-stormcode-key storm-id) nimbus))
    +        current-replication-count-conf (atom (get-blob-replication-count (master-stormconf-key storm-id) nimbus))
    +        total-wait-time (atom 0)]
    +    (log-message "wait for desired replication" "count"
    +      min-replication-count "wait-time" max-replication-wait-time
    +      "code" @current-replication-count-code
    +      "conf" @current-replication-count-conf
    +      "jar" @current-replication-count-jar
    +      "replication count" (get-blob-replication-count (master-stormconf-key storm-id) nimbus))
    +  (if (:blob-store nimbus)
    +    (while (and (> min-replication-count @current-replication-count-jar)
    +                (> min-replication-count @current-replication-count-code)
    +                (> min-replication-count @current-replication-count-conf)
                  (or (= -1 max-replication-wait-time)
                    (< @total-wait-time max-replication-wait-time)))
             (sleep-secs 1)
             (log-debug "waiting for desired replication to be achieved.
               min-replication-count = " min-replication-count  " max-replication-wait-time = " max-replication-wait-time
    -          "current-replication-count = " @current-replication-count " total-wait-time " @total-wait-time)
    +          (if (not (local-mode? conf))"current-replication-count for jar key = " @current-replication-count-jar)
    +          "current-replication-count for code key = " @current-replication-count-code
    +          "current-replication-count for conf key = " @current-replication-count-conf
    +          " total-wait-time " @total-wait-time)
             (swap! total-wait-time inc)
    -        (reset! current-replication-count  (.getReplicationCount (:code-distributor nimbus) storm-id))))
    -  (if (< min-replication-count @current-replication-count)
    -    (log-message "desired replication count "  min-replication-count " achieved,
    -      current-replication-count" @current-replication-count)
    -    (log-message "desired replication count of "  min-replication-count " not achieved but we have hit the max wait time "
    -      max-replication-wait-time " so moving on with replication count = " @current-replication-count)
    -    )))
    -
    -(defn- read-storm-topology [conf storm-id]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -    (Utils/deserialize
    -      (FileUtils/readFileToByteArray
    -        (File. (master-stormcode-path stormroot))
    -        ) StormTopology)))
    +        (if (not (local-mode? conf))(reset! current-replication-count-conf  (get-blob-replication-count (master-stormconf-key storm-id))))
    +          (reset! current-replication-count-code  (get-blob-replication-count (master-stormcode-key storm-id)))
    +          (reset! current-replication-count-jar  (get-blob-replication-count (master-stormjar-key storm-id)))))
    +  (if (and (< min-replication-count @current-replication-count-conf)
    +           (< min-replication-count @current-replication-count-code)
    +           (< min-replication-count @current-replication-count-jar))
    +      (log-message "desired replication count of "  min-replication-count " not achieved but we have hit the max wait time "
    +      max-replication-wait-time " so moving on with replication count for conf key = " @current-replication-count-conf
    +      " for code key = " @current-replication-count-code "for jar key = " @current-replication-count-jar)
    +      (log-message "desired replication count "  min-replication-count " achieved,
    +        current-replication-count for conf key " @current-replication-count-conf ",
    +        current-replication-count for code key = " @current-replication-count-code ",
    +        current-replication-count for jar key = " @current-replication-count-jar))))
    +
    +(defn- read-storm-topology-as-nimbus [storm-id blob-store]
    +  (Utils/deserialize
    +    (.readBlob blob-store (master-stormcode-key storm-id) (get-nimbus-subject)) StormTopology))
     
     (declare compute-executor->component)
     
    +(defn- get-nimbus-subject []
    --- End diff --
    
    I think this was already defined some place else.  Could we please combine the two implementations?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45759085
  
    --- Diff: storm-core/src/jvm/backtype/storm/Config.java ---
    @@ -980,6 +980,126 @@
         public static final String SUPERVISOR_SLOTS_PORTS = "supervisor.slots.ports";
     
         /**
    +     * What blobstore implementation the supervisor should use.
    +     */
    +    @isString
    +    public static final String SUPERVISOR_BLOBSTORE = "supervisor.blobstore.class";
    +    public static final Object SUPERVISOR_BLOBSTORE_SCHEMA = String.class;
    +
    +    /**
    +     * 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.
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String SUPERVISOR_LOCALIZER_CACHE_TARGET_SIZE_MB = "supervisor.localizer.cache.target.size.mb";
    +
    +    /**
    +     * The distributed cache cleanup interval. Controls how often it scans to attempt to cleanup
    +     * anything over the cache target size.
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String SUPERVISOR_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS = "supervisor.localizer.cleanup.interval.ms";
    +
    +    /**
    +     * What blobstore implementation the storm client should use.
    +     */
    +    @isString
    +    public static final String CLIENT_BLOBSTORE = "client.blobstore.class";
    +
    +    /**
    +     * What blobstore download parallelism the supervisor should use.
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String SUPERVISOR_BLOBSTORE_DOWNLOAD_THREAD_COUNT = "supervisor.blobstore.download.thread.count";
    +
    +    /**
    +     * What blobstore download parallelism the supervisor should use.
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String SUPERVISOR_BLOBSTORE_DOWNLOAD_MAX_RETRIES = "supervisor.blobstore.download.max_retries";
    +
    +    /**
    +     * The blobstore super user has all read/write/admin permissions to all blobs - user running
    +     * the blobstore.
    +     */
    +    @isString
    +    public static final String BLOBSTORE_SUPERUSER = "blobstore.superuser";
    +
    +    /**
    +     * What directory to use for the blobstore. The directory is expected to be an
    +     * absolute path when using HDFS blobstore, for LocalFsBlobStore it could be either
    +     * absolute or relative.
    +     */
    +    @isString
    +    public static final String BLOBSTORE_DIR = "blobstore.dir";
    +
    +    /**
    +     * What buffer size to use for the blobstore uploads.
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String STORM_BLOBSTORE_INPUTSTREAM_BUFFER_SIZE_BYTES = "storm.blobstore.inputstream.buffer.size.bytes";
    --- End diff --
    
    There is also a TOPOLOGY_BLOBSTORE_MIN_REPLICATION_FACTOR which specifies the amount of replication a blob has to undergo before the topology is launched, but on the other hand for HDFS blobstore we have to set the replication factor for the blob on a HDFS level. I could go ahead with HDFS_BLOBSTORE* but that would represent HDFS config rather than a STORM config. Welcome for suggestions


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45425945
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/ClientBlobStore.java ---
    @@ -0,0 +1,69 @@
    +/**
    + * 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.Config;
    +import backtype.storm.daemon.Shutdownable;
    +import backtype.storm.generated.AccessControl;
    +import backtype.storm.generated.AccessControlType;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.utils.NimbusClient;
    +import backtype.storm.utils.Utils;
    +
    +import java.util.Iterator;
    +import java.util.Map;
    +
    +
    +public abstract class ClientBlobStore implements Shutdownable {
    +  protected Map conf;
    +
    +  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;
    --- End diff --
    
    All implementations of `watchBlob` and `stopWatchingBlob` either do nothing or throw RuntimeException.  Should we remove these for now?  If we have a useful implementation, we can add them back later.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45421440
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -732,25 +863,89 @@
      (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
       (worker-launcher-and-wait conf (storm-conf TOPOLOGY-SUBMITTER-USER) ["code-dir" dir] :log-prefix (str "setup conf for " dir))))
     
    +(defn setup-blob-permission [conf storm-conf path]
    +  (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
    +    (worker-launcher-and-wait conf (storm-conf TOPOLOGY-SUBMITTER-USER) ["blob" path] :log-prefix (str "setup blob permissions for " path))))
    +
    +(defn setup-storm-code-dir [conf storm-conf dir]
    +  (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
    +    (worker-launcher-and-wait conf (storm-conf TOPOLOGY-SUBMITTER-USER) ["code-dir" dir] :log-prefix (str "setup conf for " dir))))
    --- End diff --
    
    Addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by kishorvpatil <gi...@git.apache.org>.
Github user kishorvpatil commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43821424
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/ClientBlobStore.java ---
    @@ -0,0 +1,63 @@
    +/**
    + * 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.Config;
    +import backtype.storm.daemon.Shutdownable;
    +import backtype.storm.generated.*;
    --- End diff --
    
    explicit import


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by unsleepy22 <gi...@git.apache.org>.
Github user unsleepy22 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46018545
  
    --- Diff: pom.xml ---
    @@ -469,7 +475,11 @@
                     <artifactId>curator-client</artifactId>
                     <version>${curator.version}</version>
                 </dependency>
    -
    +            <dependency>
    +                <groupId>org.apache.curator</groupId>
    +                <artifactId>curator-test</artifactId>
    +                <version>${curator.version}</version>
    --- End diff --
    
    scope=test?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43765672
  
    --- Diff: pom.xml ---
    @@ -646,11 +646,92 @@
                     <version>${thrift.version}</version>
                     <scope>compile</scope>
                 </dependency>
    +			<!-- used by examples/storm-starter -->
    +		    <dependency>
    +		      <groupId>junit</groupId>
    +		      <artifactId>junit</artifactId>
    +		      <version>4.11</version>
    +		      <scope>test</scope>
    +		    </dependency>
    --- End diff --
    
    The indentation here is off


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43786013
  
    --- Diff: storm-core/src/clj/backtype/storm/cluster.clj ---
    @@ -408,13 +416,14 @@
               (swap! assignment-version-callback assoc storm-id callback))
             (get-version cluster-state (assignment-path storm-id) (not-nil? callback)))
     
    -      (code-distributor
    +      ;; blobstore state
    +      (blobstore
             [this callback]
    -        (when callback
    -          (reset! code-distributor-callback callback))
    +        (when callback)
    --- End diff --
    
    oops fixed it


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45269639
  
    --- Diff: storm-core/src/jvm/backtype/storm/Config.java ---
    @@ -980,6 +980,126 @@
         public static final String SUPERVISOR_SLOTS_PORTS = "supervisor.slots.ports";
     
         /**
    +     * What blobstore implementation the supervisor should use.
    +     */
    +    @isString
    +    public static final String SUPERVISOR_BLOBSTORE = "supervisor.blobstore.class";
    +    public static final Object SUPERVISOR_BLOBSTORE_SCHEMA = String.class;
    +
    +    /**
    +     * 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.
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String SUPERVISOR_LOCALIZER_CACHE_TARGET_SIZE_MB = "supervisor.localizer.cache.target.size.mb";
    +
    +    /**
    +     * The distributed cache cleanup interval. Controls how often it scans to attempt to cleanup
    +     * anything over the cache target size.
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String SUPERVISOR_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS = "supervisor.localizer.cleanup.interval.ms";
    +
    +    /**
    +     * What blobstore implementation the storm client should use.
    +     */
    +    @isString
    +    public static final String CLIENT_BLOBSTORE = "client.blobstore.class";
    +
    +    /**
    +     * What blobstore download parallelism the supervisor should use.
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String SUPERVISOR_BLOBSTORE_DOWNLOAD_THREAD_COUNT = "supervisor.blobstore.download.thread.count";
    +
    +    /**
    +     * What blobstore download parallelism the supervisor should use.
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String SUPERVISOR_BLOBSTORE_DOWNLOAD_MAX_RETRIES = "supervisor.blobstore.download.max_retries";
    +
    +    /**
    +     * The blobstore super user has all read/write/admin permissions to all blobs - user running
    +     * the blobstore.
    +     */
    +    @isString
    +    public static final String BLOBSTORE_SUPERUSER = "blobstore.superuser";
    +
    +    /**
    +     * What directory to use for the blobstore. The directory is expected to be an
    +     * absolute path when using HDFS blobstore, for LocalFsBlobStore it could be either
    +     * absolute or relative.
    +     */
    +    @isString
    +    public static final String BLOBSTORE_DIR = "blobstore.dir";
    +
    +    /**
    +     * What buffer size to use for the blobstore uploads.
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String STORM_BLOBSTORE_INPUTSTREAM_BUFFER_SIZE_BYTES = "storm.blobstore.inputstream.buffer.size.bytes";
    +
    +    /**
    +     * Enable the blobstore cleaner. Certain blobstores may only want to run the cleaner
    +     * on one daemon. Currently Nimbus handles setting this.
    +     */
    +    @isBoolean
    +    public static final String BLOBSTORE_CLEANUP_ENABLE = "blobstore.cleanup.enable";
    +
    +    /**
    +     * principal for nimbus/supervisor to use to access secure hdfs for the blobstore.
    +     */
    +    @isString
    +    public static final String BLOBSTORE_HDFS_PRINCIPAL = "blobstore.hdfs.principal";
    +
    +    /**
    +     * keytab for nimbus/supervisor to use to access secure hdfs for the blobstore.
    +     */
    +    @isString
    +    public static final String BLOBSTORE_HDFS_KEYTAB = "blobstore.hdfs.keytab";
    +
    +    /**
    +     *  Set replication factor for a blob in HDFS Blobstore Implementation
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String STORM_BLOBSTORE_REPLICATION_FACTOR = "storm.blobstore.replication.factor";
    --- End diff --
    
    Why do some of these new configs start with `storm.` and others don't?  There might be a reason for this I am not thinking of at the moment.  If not, it would be good to standardize these.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43766138
  
    --- Diff: storm-core/pom.xml ---
    @@ -195,6 +206,23 @@
                 <scope>compile</scope>
             </dependency>
             <dependency>
    +            <groupId>org.apache.hadoop</groupId>
    +            <artifactId>hadoop-client</artifactId>
    +            <scope>provided</scope>
    +        </dependency>
    +        <dependency>
    +            <groupId>org.apache.hadoop</groupId>
    +            <artifactId>hadoop-common</artifactId>
    +            <type>test-jar</type>
    +            <scope>test</scope>
    +        </dependency>
    +        <dependency>
    +            <groupId>org.apache.hadoop</groupId>
    +            <artifactId>hadoop-hdfs</artifactId>
    +            <type>test-jar</type>
    +            <scope>test</scope>
    +        </dependency>
    +        <dependency>
    --- End diff --
    
    Do we want to move the HDFS implementation and its dependencies to the external package like how we had for the original HA work?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45241587
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -326,16 +330,60 @@
                                              (log-error t "Error when processing event")
                                              (exit-process! 20 "Error when processing an event")
                                              ))
    +   :blob-update-timer (mk-timer :kill-fn (fn [t]
    +                                           (log-error t "Error when processing event")
    +                                           (exit-process! 20 "Error when processing a event")))
    --- End diff --
    
    Can we set `:timer-name` here in addition to the `:kill-fn`?
    It might also be helpful for debugging to give a name here to the anonymous `fn`, as we have seen some mysterious hangs while logging.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45099492
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -390,53 +444,98 @@
           [(.getNodeId slot) (.getPort slot)]
           )))
     
    +(defn- get-version-for-key [key nimbus-host-port-info conf]
    +  (let [version (KeyVersion. key nimbus-host-port-info)]
    +    (.getKeyVersion version conf)))
    +
    +(defn get-key-seq-from-blob-store [blob-store]
    +  (let [key-iter (.listKeys blob-store nimbus-subject)]
    +    (iterator-seq key-iter)))
    +
     (defn- setup-storm-code [nimbus conf storm-id tmp-jar-location storm-conf topology]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -   (log-message "nimbus file location:" stormroot)
    -   (FileUtils/forceMkdir (File. stormroot))
    -   (FileUtils/cleanDirectory (File. stormroot))
    -   (setup-jar conf tmp-jar-location stormroot)
    -   (FileUtils/writeByteArrayToFile (File. (master-stormcode-path stormroot)) (Utils/serialize topology))
    -   (FileUtils/writeByteArrayToFile (File. (master-stormconf-path stormroot)) (Utils/toCompressedJsonConf storm-conf))
    -   (if (:code-distributor nimbus) (.upload (:code-distributor nimbus) stormroot storm-id))
    -   ))
    +  (let [subject (get-subject)
    +        storm-cluster-state (:storm-cluster-state nimbus)
    +        blob-store (:blob-store nimbus)
    +        jar-key (master-stormjar-key storm-id)
    +        code-key (master-stormcode-key storm-id)
    +        conf-key (master-stormconf-key storm-id)
    +        nimbus-host-port-info (:nimbus-host-port-info nimbus)]
    +    (when tmp-jar-location ;;in local mode there is no jar
    +      (.createBlob blob-store jar-key (FileInputStream. tmp-jar-location) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +      (if (instance? LocalFsBlobStore blob-store)
    +        (.setup-blobstore! storm-cluster-state jar-key nimbus-host-port-info (get-version-for-key jar-key nimbus-host-port-info conf))))
    +    (.createBlob blob-store conf-key (Utils/toCompressedJsonConf storm-conf) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state conf-key nimbus-host-port-info (get-version-for-key conf-key nimbus-host-port-info conf)))
    +    (.createBlob blob-store code-key (Utils/serialize topology) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state code-key nimbus-host-port-info (get-version-for-key code-key nimbus-host-port-info conf)))))
    +
    +(defn- read-storm-topology [storm-id blob-store]
    +  (Utils/deserialize
    +    (.readBlob blob-store (master-stormcode-key storm-id) (get-subject)) StormTopology))
    +
    +(defn- get-blob-replication-count [blob-key nimbus]
    +  (if (:blob-store nimbus)
    +          (-> (:blob-store nimbus)
    +            (.getBlobReplication  blob-key nimbus-subject))))
     
     (defn- wait-for-desired-code-replication [nimbus conf storm-id]
       (let [min-replication-count (conf TOPOLOGY-MIN-REPLICATION-COUNT)
             max-replication-wait-time (conf TOPOLOGY-MAX-REPLICATION-WAIT-TIME-SEC)
    -        total-wait-time (atom 0)
    -        current-replication-count (atom (if (:code-distributor nimbus) (.getReplicationCount (:code-distributor nimbus) storm-id) 0))]
    -  (if (:code-distributor nimbus)
    -    (while (and (> min-replication-count @current-replication-count)
    -             (or (= -1 max-replication-wait-time)
    -               (< @total-wait-time max-replication-wait-time)))
    +        current-replication-count-jar (if (not (local-mode? conf))
    +                                        (atom (get-blob-replication-count (master-stormjar-key storm-id) nimbus))
    +                                        (atom min-replication-count))
    +        current-replication-count-code (atom (get-blob-replication-count (master-stormcode-key storm-id) nimbus))
    +        current-replication-count-conf (atom (get-blob-replication-count (master-stormconf-key storm-id) nimbus))
    +        total-wait-time (atom 0)]
    +    (if (:blob-store nimbus)
    +      (while (and
    +               (or (> min-replication-count @current-replication-count-jar)
    +                   (> min-replication-count @current-replication-count-code)
    +                   (> min-replication-count @current-replication-count-conf))
    +               (or (= -1 max-replication-wait-time)
    +                   (< @total-wait-time max-replication-wait-time)))
    --- End diff --
    
    Can we test for `(neg? max-replication-wait-time)`, as per my previous comment, or is there a reason we want to check specifically for `-1`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46094374
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/NimbusBlobStore.java ---
    @@ -0,0 +1,408 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.BeginDownloadResult;
    +import backtype.storm.generated.ListBlobsResult;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.utils.NimbusClient;
    +import backtype.storm.utils.Utils;
    +import org.apache.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.nio.ByteBuffer;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +
    +public class NimbusBlobStore extends ClientBlobStore {
    +  private static final Logger LOG = LoggerFactory.getLogger(NimbusBlobStore.class);
    +
    +  public class NimbusKeyIterator implements Iterator<String> {
    +    private ListBlobsResult listBlobs = null;
    +    private int offset = 0;
    +    private boolean eof = false;
    +    
    +    public NimbusKeyIterator(ListBlobsResult listBlobs) {
    +      this.listBlobs = listBlobs;
    +      this.eof = (listBlobs.get_keys_size() == 0);
    +    }
    +    
    +    private boolean isCacheEmpty() {
    +      return listBlobs.get_keys_size() <= offset;
    +    }
    +
    +    private void readMore() {
    +      if (!eof) {
    +        try {
    +          offset = 0;
    +          synchronized(client) {
    +            listBlobs = client.getClient().listBlobs(listBlobs.get_session());
    +          }
    +          if (listBlobs.get_keys_size() == 0) {
    +            eof = true;
    +          }
    +        } catch (TException e) {
    +          throw new RuntimeException(e);
    +        }
    +      }
    +    }
    +    
    +    @Override
    +    public synchronized boolean hasNext() {
    +      if (isCacheEmpty()) {
    +        readMore();
    +      }
    +      return !eof;
    +    }
    +
    +    @Override
    +    public synchronized String next() {
    +      if (!hasNext()) {
    +        throw new NoSuchElementException();
    +      }
    +      String ret = listBlobs.get_keys().get(offset);
    +      offset++;
    +      return ret;
    +    }
    +    
    +    @Override
    +    public void remove() {
    +      throw new UnsupportedOperationException("Delete Not Supported");
    +    }
    +  }
    +
    +  public class NimbusDownloadInputStream extends InputStreamWithMeta {
    +    private BeginDownloadResult beginBlobDownload;
    +    private byte[] buffer = null;
    +    private int offset = 0;
    +    private int end = 0;
    +    private boolean eof = false;
    +
    +    public NimbusDownloadInputStream(BeginDownloadResult beginBlobDownload) {
    +      this.beginBlobDownload = beginBlobDownload;
    +    }
    +
    +    @Override
    +    public long getVersion() throws IOException {
    +      return beginBlobDownload.get_version();
    +    }
    +
    +    @Override
    +    public synchronized int read() throws IOException {
    +      if (isEmpty()) {
    +        readMore();
    +        if (eof) {
    +          return -1;
    +        }
    +      }
    +      int length = Math.min(1, available());
    +      if (length == 0) {
    +        return -1;
    +      }
    +      int ret = buffer[offset];
    +      offset += length;
    +      return ret;
    +    }
    +    
    +    @Override 
    +    public synchronized int read(byte[] b, int off, int len) throws IOException {
    +      if (isEmpty()) {
    +        readMore();
    +        if (eof) {
    +          return -1;
    +        }
    +      }
    +      int length = Math.min(len, available());
    +      System.arraycopy(buffer, offset, b, off, length);
    +      offset += length;
    +      return length;
    +    }
    +    
    +    private boolean isEmpty() {
    +      return buffer == null || offset >= end;
    +    }
    +    
    +    private void readMore() {
    +      if (!eof) {
    +        try {
    +          ByteBuffer buff;
    +          synchronized(client) {
    +            buff = client.getClient().downloadBlobChunk(beginBlobDownload.get_session());
    +          }
    +          buffer = buff.array();
    +          offset = buff.arrayOffset() + buff.position();
    +          int length = buff.remaining();
    +          end = offset + length;
    +          if (length == 0) {
    +            eof = true;
    +          }
    +        } catch (TException e) {
    +          throw new RuntimeException(e);
    +        }
    +      }
    +    }
    +    
    +    @Override 
    +    public synchronized int read(byte[] b) throws IOException {
    +      return read(b, 0, b.length);
    +    }
    +    
    +    @Override
    +    public synchronized int available() {
    +      return buffer == null ? 0 : (end - offset);
    +    }
    +
    +    @Override
    +    public long getFileLength() {
    +        return beginBlobDownload.get_data_size();
    +    }
    +  }
    +
    +  public class NimbusUploadAtomicOutputStream extends AtomicOutputStream {
    +    private String session;
    +    private int maxChunkSize = 4096;
    +    private String key;
    +
    +    public NimbusUploadAtomicOutputStream(String session, int bufferSize, String key) {
    +      this.session = session;
    +      this.maxChunkSize = bufferSize;
    +      this.key = key;
    +    }
    +
    +    @Override
    +    public void cancel() throws IOException {
    +      try {
    +        synchronized(client) {
    +          client.getClient().cancelBlobUpload(session);
    +        }
    +      } catch (TException e) {
    +        throw new RuntimeException(e);
    +      }
    +    }
    +
    +    @Override
    +    public void write(int b) throws IOException {
    +      try {
    +        synchronized(client) {
    +          client.getClient().uploadBlobChunk(session, ByteBuffer.wrap(new byte[] {(byte)b}));
    +        }
    +      } catch (TException e) {
    +        throw new RuntimeException(e);
    +      }
    +    }
    +    
    +    @Override
    +    public void write(byte []b) throws IOException {
    +      write(b, 0, b.length);
    +    }
    +    
    +    @Override
    +    public void write(byte []b, int offset, int len) throws IOException {
    +      try {
    +        int end = offset + len;
    +        for (int realOffset = offset; realOffset < end; realOffset += maxChunkSize) {
    +          int realLen = Math.min(end - realOffset, maxChunkSize);
    +          LOG.debug("Writing {} bytes of {} remaining",realLen,(end-realOffset));
    +          synchronized(client) {
    +            client.getClient().uploadBlobChunk(session, ByteBuffer.wrap(b, realOffset, realLen));
    +          }
    +        }
    +      } catch (TException e) {
    +        throw new RuntimeException(e);
    +      }
    +    }
    +    
    +    @Override
    +    public void close() throws IOException {
    +      try {
    +        synchronized(client) {
    +          client.getClient().finishBlobUpload(session);
    +          client.getClient().createStateInZookeeper(key);
    +        }
    +      } catch (TException e) {
    +        throw new RuntimeException(e);
    +      }
    +    }
    +  }
    +
    +  private NimbusClient client;
    +  private int bufferSize = 4096;
    +
    +  @Override
    +  public void prepare(Map conf) {
    +    this.client = NimbusClient.getConfiguredClient(conf);
    +    if (conf != null) {
    +        this.bufferSize = Utils.getInt(conf.get(Config.STORM_BLOBSTORE_INPUTSTREAM_BUFFER_SIZE_BYTES), bufferSize);
    +    }
    +  }
    +
    +  @Override
    +  protected AtomicOutputStream createBlobToExtend(String key, SettableBlobMeta meta)
    +      throws AuthorizationException, KeyAlreadyExistsException {
    +    try {
    +      synchronized(client) {
    +        return new NimbusUploadAtomicOutputStream(client.getClient().beginCreateBlob(key, meta), this.bufferSize, key);
    +      }
    +    } catch (AuthorizationException | KeyAlreadyExistsException exp) {
    --- End diff --
    
    Here Authorization and KeyAlreadyExists are TExceptions as per thrift. Therefore, we will not be able to catch specfic exceptions and log them higher up in the stack


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45272038
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStore.java ---
    @@ -0,0 +1,446 @@
    +/**
    + * 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 java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Set;
    +import java.util.regex.Pattern;
    +
    +import javax.security.auth.Subject;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import backtype.storm.daemon.Shutdownable;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +
    +/**
    + * Provides a way to store blobs that can be downloaded.
    + * Blobs must be able to be uploaded and listed from Nimbus,
    + * and downloaded from the Supervisors. It is a key value based
    + * store. Key being a string and value being the blob data.
    + *
    + * ACL checking must take place against the provided subject.
    + * If the blob store does not support Security it must validate
    + * that all ACLs set are always WORLD, everything.
    + *
    + * The users can upload their blobs through the blob store command
    + * line utility. The command line utilty also allows us to update,
    + * delete.
    + *
    + * Modifying the replication factor only works for HdfsBlobStore
    + * as for the LocalFsBlobStore the replication is dependent on
    + * the number of Nimbodes available.
    + */
    +public abstract class BlobStore implements Shutdownable {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStore.class);
    +  private static final Pattern KEY_PATTERN = Pattern.compile("^[\\w \\t\\.:_-]+$");
    +  protected static final String BASE_BLOBS_DIR_NAME = "blobs";
    +
    +  /**
    +   * Allows us to initialize the blob store
    +   * @param conf
    +   * @param baseDir
    +   */
    +  public abstract void prepare(Map conf, String baseDir, NimbusInfo nimbusInfo);
    +
    +  /**
    +   * Creates the blob.
    +   * @param key Key for the blob.
    +   * @param meta Metadata which contains the acls information
    +   * @param who Is the subject creating the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyAlreadyExistsException
    +   */
    +  public abstract AtomicOutputStream createBlob(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException;
    +
    +  /**
    +   * Updates the blob data.
    +   * @param key Key for the blob.
    +   * @param who Is the subject creating the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract AtomicOutputStream updateBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Gets the current version of metadata for a blob
    +   * to be viewed by the user or downloaded by the supervisor.
    +   * @param key Key for the blob.
    +   * @param who Is the subject creating the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract ReadableBlobMeta getBlobMeta(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Sets the metadata with renewed acls for the blob.
    +   * @param key Key for the blob.
    +   * @param meta Metadata which contains the updated
    +   * acls information.
    +   * @param who Is the subject creating the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract void setBlobMeta(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Deletes the blob data and metadata.
    +   * @param key Key for the blob.
    +   * @param who Is the subject creating the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract void deleteBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Gets the current version of metadata for a blob
    +   * to be viewed by the user or downloaded by the supervisor.
    +   * @param key Key for the blob.
    +   * @param who Is the subject creating the blob.
    +   * @return InputStreamWithMeta has the additional
    +   * file length and version information.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract InputStreamWithMeta getBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    --- End diff --
    
    Wrong javadoc


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45427929
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/KeyVersion.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * 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.zookeeper.CreateMode;
    +import org.apache.zookeeper.ZooDefs;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.TreeSet;
    +import java.util.Map;
    +import java.util.List;
    +
    +/**
    + * Class hands over the version of the key to be stored within the zookeeper
    + */
    +public class KeyVersion {
    +  private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
    +  private final String BLOBSTORE_SUBTREE="/blobstore";
    +  private final String BLOBSTORE_KEY_COUNTER_SUBTREE="/blobstorekeycounter";
    +  private String key;
    +  private NimbusInfo nimbusInfo;
    +
    +  public KeyVersion(String key, NimbusInfo nimbusInfo) {
    +    this.key = key;
    +    this.nimbusInfo = nimbusInfo;
    +  }
    +
    +  public int getKeyVersion(Map conf) {
    +    TreeSet<Integer> versions = new TreeSet<Integer>();
    +    CuratorFramework zkClient = Utils.createZKClient(conf);
    +    try {
    +      // Key has not been created yet and it is the first time it is being created
    +      if(zkClient.checkExists().forPath(BLOBSTORE_SUBTREE + "/" + key) == null) {
    +        zkClient.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT)
    +                .withACL(ZooDefs.Ids.OPEN_ACL_UNSAFE).forPath(BLOBSTORE_KEY_COUNTER_SUBTREE + "/" + key + "/" + 1);
    +        return 1;
    +      }
    +
    +      // When all nimbodes go down and one or few of them come up
    +      // Unfortunately there might not be an exact way to know which one contains the most updated blob
    +      // if all go down which is unlikely. Hence there might be a need to update the blob if all go down
    +      List<String> stateInfoList = zkClient.getChildren().forPath(BLOBSTORE_SUBTREE + "/" + key);
    +      LOG.debug("stateInfoList {} stateInfoList {}", stateInfoList.size(), stateInfoList);
    +      if(stateInfoList.isEmpty()) {
    +        return getKeyVersionCounterValue(zkClient, key);
    +      }
    +
    +      LOG.debug("stateInfoSize {}", stateInfoList.size());
    +      // In all other cases check for the latest version on the nimbus and assign the version
    +      // check if all are have same version, if not assign the highest version
    +      for (String stateInfo:stateInfoList) {
    +        versions.add(Integer.parseInt(Utils.normalizeVersionInfo(stateInfo)[1]));
    +      }
    +
    +      int currentCounter = getKeyVersionCounterValue(zkClient, key);
    +      // This condition returns version when a nimbus crashes and comes up
    +      if (!checkIfStateContainsCurrentNimbusHost(stateInfoList, nimbusInfo) && !nimbusInfo.isLeader()) {
    +        if (versions.last() < currentCounter) {
    +          return currentCounter;
    +        } else {
    +          return currentCounter - 1;
    +        }
    +      }
    +      // Condition checks for an update scenario
    +      if (stateInfoList.size() >= 1 && versions.size() == 1) {
    +        if (versions.first() < getKeyVersionCounterValue(zkClient, key)) {
    +          incrementCounter(zkClient, key, currentCounter);
    +          return currentCounter + 1;
    +        } else {
    +          incrementCounter(zkClient, key, currentCounter);
    +          return versions.first() + 1;
    +        }
    +      }
    +    } catch(Exception e) {
    +      LOG.error("Exception {}", e);
    +    } finally {
    +      if (zkClient != null) {
    +        zkClient.close();
    +      }
    +    }
    +    return versions.last();
    +  }
    +
    +  public boolean checkIfStateContainsCurrentNimbusHost(List<String> stateInfoList, NimbusInfo nimbusInfo) {
    --- End diff --
    
    `private`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45100258
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -390,53 +444,98 @@
           [(.getNodeId slot) (.getPort slot)]
           )))
     
    +(defn- get-version-for-key [key nimbus-host-port-info conf]
    +  (let [version (KeyVersion. key nimbus-host-port-info)]
    +    (.getKeyVersion version conf)))
    +
    +(defn get-key-seq-from-blob-store [blob-store]
    +  (let [key-iter (.listKeys blob-store nimbus-subject)]
    +    (iterator-seq key-iter)))
    +
     (defn- setup-storm-code [nimbus conf storm-id tmp-jar-location storm-conf topology]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -   (log-message "nimbus file location:" stormroot)
    -   (FileUtils/forceMkdir (File. stormroot))
    -   (FileUtils/cleanDirectory (File. stormroot))
    -   (setup-jar conf tmp-jar-location stormroot)
    -   (FileUtils/writeByteArrayToFile (File. (master-stormcode-path stormroot)) (Utils/serialize topology))
    -   (FileUtils/writeByteArrayToFile (File. (master-stormconf-path stormroot)) (Utils/toCompressedJsonConf storm-conf))
    -   (if (:code-distributor nimbus) (.upload (:code-distributor nimbus) stormroot storm-id))
    -   ))
    +  (let [subject (get-subject)
    +        storm-cluster-state (:storm-cluster-state nimbus)
    +        blob-store (:blob-store nimbus)
    +        jar-key (master-stormjar-key storm-id)
    +        code-key (master-stormcode-key storm-id)
    +        conf-key (master-stormconf-key storm-id)
    +        nimbus-host-port-info (:nimbus-host-port-info nimbus)]
    +    (when tmp-jar-location ;;in local mode there is no jar
    +      (.createBlob blob-store jar-key (FileInputStream. tmp-jar-location) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +      (if (instance? LocalFsBlobStore blob-store)
    +        (.setup-blobstore! storm-cluster-state jar-key nimbus-host-port-info (get-version-for-key jar-key nimbus-host-port-info conf))))
    +    (.createBlob blob-store conf-key (Utils/toCompressedJsonConf storm-conf) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state conf-key nimbus-host-port-info (get-version-for-key conf-key nimbus-host-port-info conf)))
    +    (.createBlob blob-store code-key (Utils/serialize topology) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state code-key nimbus-host-port-info (get-version-for-key code-key nimbus-host-port-info conf)))))
    +
    +(defn- read-storm-topology [storm-id blob-store]
    +  (Utils/deserialize
    +    (.readBlob blob-store (master-stormcode-key storm-id) (get-subject)) StormTopology))
    +
    +(defn- get-blob-replication-count [blob-key nimbus]
    +  (if (:blob-store nimbus)
    +          (-> (:blob-store nimbus)
    +            (.getBlobReplication  blob-key nimbus-subject))))
     
     (defn- wait-for-desired-code-replication [nimbus conf storm-id]
       (let [min-replication-count (conf TOPOLOGY-MIN-REPLICATION-COUNT)
             max-replication-wait-time (conf TOPOLOGY-MAX-REPLICATION-WAIT-TIME-SEC)
    -        total-wait-time (atom 0)
    -        current-replication-count (atom (if (:code-distributor nimbus) (.getReplicationCount (:code-distributor nimbus) storm-id) 0))]
    -  (if (:code-distributor nimbus)
    -    (while (and (> min-replication-count @current-replication-count)
    -             (or (= -1 max-replication-wait-time)
    -               (< @total-wait-time max-replication-wait-time)))
    +        current-replication-count-jar (if (not (local-mode? conf))
    +                                        (atom (get-blob-replication-count (master-stormjar-key storm-id) nimbus))
    +                                        (atom min-replication-count))
    +        current-replication-count-code (atom (get-blob-replication-count (master-stormcode-key storm-id) nimbus))
    +        current-replication-count-conf (atom (get-blob-replication-count (master-stormconf-key storm-id) nimbus))
    +        total-wait-time (atom 0)]
    +    (if (:blob-store nimbus)
    +      (while (and
    +               (or (> min-replication-count @current-replication-count-jar)
    +                   (> min-replication-count @current-replication-count-code)
    +                   (> min-replication-count @current-replication-count-conf))
    +               (or (= -1 max-replication-wait-time)
    +                   (< @total-wait-time max-replication-wait-time)))
             (sleep-secs 1)
             (log-debug "waiting for desired replication to be achieved.
               min-replication-count = " min-replication-count  " max-replication-wait-time = " max-replication-wait-time
    -          "current-replication-count = " @current-replication-count " total-wait-time " @total-wait-time)
    +          (if (not (local-mode? conf))"current-replication-count for jar key = " @current-replication-count-jar)
    +          "current-replication-count for code key = " @current-replication-count-code
    +          "current-replication-count for conf key = " @current-replication-count-conf
    +          " total-wait-time " @total-wait-time)
             (swap! total-wait-time inc)
    -        (reset! current-replication-count  (.getReplicationCount (:code-distributor nimbus) storm-id))))
    -  (if (< min-replication-count @current-replication-count)
    -    (log-message "desired replication count "  min-replication-count " achieved,
    -      current-replication-count" @current-replication-count)
    -    (log-message "desired replication count of "  min-replication-count " not achieved but we have hit the max wait time "
    -      max-replication-wait-time " so moving on with replication count = " @current-replication-count)
    -    )))
    -
    -(defn- read-storm-topology [conf storm-id]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -    (Utils/deserialize
    -      (FileUtils/readFileToByteArray
    -        (File. (master-stormcode-path stormroot))
    -        ) StormTopology)))
    +        (if (not (local-mode? conf))
    +          (reset! current-replication-count-conf  (get-blob-replication-count (master-stormconf-key storm-id))))
    +        (reset! current-replication-count-code  (get-blob-replication-count (master-stormcode-key storm-id)))
    +        (reset! current-replication-count-jar  (get-blob-replication-count (master-stormjar-key storm-id)))))
    --- End diff --
    
    Re-posting since my previous comment was not addressed:
    
    Here we are calling `get-blob-replication-count` three times with one argument, while the arity of the funciton is 2.  I think the missing argument here might be `nimbus`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by unsleepy22 <gi...@git.apache.org>.
Github user unsleepy22 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46023183
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStoreUtils.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.security.auth.NimbusPrincipal;
    +import backtype.storm.utils.NimbusClient;
    +import backtype.storm.utils.Utils;
    +import backtype.storm.utils.ZookeeperAuthInfo;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.thrift.transport.TTransportException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class BlobStoreUtils {
    +    private static final String BLOBSTORE_SUBTREE="/blobstore";
    +    private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
    +
    +    public static CuratorFramework createZKClient(Map conf) {
    +        List<String> zkServers = (List<String>) conf.get(Config.STORM_ZOOKEEPER_SERVERS);
    +        Object port = conf.get(Config.STORM_ZOOKEEPER_PORT);
    +        ZookeeperAuthInfo zkAuthInfo = new ZookeeperAuthInfo(conf);
    +        CuratorFramework zkClient = Utils.newCurator(conf, zkServers, port, (String) conf.get(Config.STORM_ZOOKEEPER_ROOT), zkAuthInfo);
    +        zkClient.start();
    +        return zkClient;
    +    }
    +
    +    public static Subject getNimbusSubject() {
    +        Subject subject = new Subject();
    +        subject.getPrincipals().add(new NimbusPrincipal());
    +        return subject;
    +    }
    +
    +    // Normalize state
    +    public static BlobKeySequenceInfo normalizeNimbusHostPortSequenceNumberInfo(String nimbusSeqNumberInfo) {
    +        BlobKeySequenceInfo keySequenceInfo = new BlobKeySequenceInfo();
    +        int lastIndex = nimbusSeqNumberInfo.lastIndexOf("-");
    +        keySequenceInfo.setNimbusHostPort(nimbusSeqNumberInfo.substring(0, lastIndex));
    +        keySequenceInfo.setSequenceNumber(nimbusSeqNumberInfo.substring(lastIndex + 1));
    +        return keySequenceInfo;
    +    }
    +
    +    // Check for latest sequence number of a key inside zookeeper and return nimbodes containing the latest sequence number
    +    public static Set<NimbusInfo> getNimbodesWithLatestSequenceNumberOfBlob(CuratorFramework zkClient, String key) throws Exception {
    +        List<String> stateInfoList = zkClient.getChildren().forPath("/blobstore/" + key);
    +        Set<NimbusInfo> nimbusInfoSet = new HashSet<NimbusInfo>();
    +        int latestSeqNumber = getLatestSequenceNumber(stateInfoList);
    +        LOG.debug("getNimbodesWithLatestSequenceNumberOfBlob stateInfo {} version {}", stateInfoList, latestSeqNumber);
    +        // Get the nimbodes with the latest version
    +        for(String state : stateInfoList) {
    +            BlobKeySequenceInfo sequenceInfo = normalizeNimbusHostPortSequenceNumberInfo(state);
    +            if (latestSeqNumber == Integer.parseInt(sequenceInfo.getSequenceNumber())) {
    +                nimbusInfoSet.add(NimbusInfo.parse(sequenceInfo.getNimbusHostPort()));
    +            }
    +        }
    +        LOG.debug("nimbusInfoList {}", nimbusInfoSet);
    +        return nimbusInfoSet;
    +    }
    +
    +    // Get sequence number details from latest sequence number of the blob
    +    public static int getLatestSequenceNumber(List<String> stateInfoList) {
    +        int seqNumber = 0;
    +        // Get latest sequence number of the blob present in the zookeeper --> possible to refactor this piece of code
    +        for (String state : stateInfoList) {
    +            BlobKeySequenceInfo sequenceInfo = normalizeNimbusHostPortSequenceNumberInfo(state);
    +            if (seqNumber < Integer.parseInt(sequenceInfo.getSequenceNumber())) {
    +                seqNumber = Integer.parseInt(sequenceInfo.getSequenceNumber());
    +                LOG.debug("Sequence Info {}", seqNumber);
    +            }
    +        }
    +        LOG.debug("Latest Sequence Number {}", seqNumber);
    +        return seqNumber;
    +    }
    +
    +    // Download missing blobs from potential nimbodes
    +    public static boolean downloadMissingBlob(Map conf, BlobStore blobStore, String key, Set<NimbusInfo> nimbusInfos)
    +            throws TTransportException {
    +        NimbusClient client = null;
    +        ReadableBlobMeta rbm = null;
    +        ClientBlobStore remoteBlobStore = null;
    +        InputStreamWithMeta in = null;
    +        Boolean isSuccess = false;
    +        LOG.debug("Download blob NimbusInfos {}", nimbusInfos);
    +        for (NimbusInfo nimbusInfo : nimbusInfos) {
    +            if(isSuccess) {
    +                break;
    +            }
    +            try {
    +                client = new NimbusClient(conf, nimbusInfo.getHost(), nimbusInfo.getPort(), null);
    +                rbm = client.getClient().getBlobMeta(key);
    +                remoteBlobStore = new NimbusBlobStore();
    +                remoteBlobStore.setClient(conf, client);
    +                in = remoteBlobStore.getBlob(key);
    +                blobStore.createBlob(key, in, rbm.get_settable(), getNimbusSubject());
    +                // if key already exists while creating the blob else update it
    +                Iterator<String> keyIterator = blobStore.listKeys();
    +                while (keyIterator.hasNext()) {
    +                    if (keyIterator.next().equals(key)) {
    +                        LOG.debug("Success creating key, {}", key);
    +                        isSuccess = true;
    +                        break;
    +                    }
    +                }
    +            } catch (IOException | AuthorizationException exception) {
    +                throw new RuntimeException(exception);
    +            } catch (KeyAlreadyExistsException kae) {
    +                LOG.info("KeyAlreadyExistsException Key: {} {}", key, kae);
    +            } catch (KeyNotFoundException knf) {
    +                // Catching and logging KeyNotFoundException because, if
    +                // there is a subsequent update and delete, the non-leader
    +                // nimbodes might throw an exception.
    +                LOG.info("KeyNotFoundException Key: {} {}", key, knf);
    +            } catch (Exception exp) {
    +                // Logging an exception while client is connecting
    +                LOG.error("Exception {}", exp);
    +            }
    +        }
    +
    +        if (!isSuccess) {
    +            LOG.error("Could not download blob with key" + key);
    +            return false;
    +        }
    +        return isSuccess;
    +    }
    +
    +    // Download updated blobs from potential nimbodes
    +    public static boolean downloadUpdatedBlob(Map conf, BlobStore blobStore, String key, Set<NimbusInfo> nimbusInfos)
    +            throws TTransportException {
    +        NimbusClient client = null;
    +        ClientBlobStore remoteBlobStore = null;
    +        InputStreamWithMeta in = null;
    +        AtomicOutputStream out = null;
    +        Boolean isSuccess = false;
    +        LOG.debug("Download blob NimbusInfos {}", nimbusInfos);
    +        for (NimbusInfo nimbusInfo : nimbusInfos) {
    +            if (isSuccess) {
    +                break;
    +            }
    +            try {
    +                client = new NimbusClient(conf, nimbusInfo.getHost(), nimbusInfo.getPort(), null);
    +                remoteBlobStore = new NimbusBlobStore();
    +                remoteBlobStore.setClient(conf, client);
    +                in = remoteBlobStore.getBlob(key);
    +                out = blobStore.updateBlob(key, getNimbusSubject());
    +                byte[] buffer = new byte[2048];
    +                int len = 0;
    +                while ((len = in.read(buffer)) > 0) {
    +                    out.write(buffer, 0, len);
    +                }
    +                if (out != null) {
    +                    out.close();
    +                }
    +                isSuccess = true;
    +            } catch (IOException | AuthorizationException exception) {
    +                throw new RuntimeException(exception);
    +            } catch (KeyNotFoundException knf) {
    +                // Catching and logging KeyNotFoundException because, if
    +                // there is a subsequent update and delete, the non-leader
    +                // nimbodes might throw an exception.
    +                LOG.info("KeyNotFoundException {}", knf);
    +            } catch (Exception exp) {
    +                // Logging an exception while client is connecting
    +                LOG.error("Exception {}", exp);
    +            }
    +        }
    +
    +        if (!isSuccess) {
    +            LOG.error("Could not update the blob with key" + key);
    +            return false;
    +        }
    +        return isSuccess;
    +    }
    +
    +    // Get the list of keys from blobstore
    +    public static List<String> getKeyListFromBlobStore(BlobStore blobStore) throws Exception {
    +        Iterator<String> keys = blobStore.listKeys();
    +        List<String> keyList = new ArrayList<String>();
    +        while (keys != null && keys.hasNext()) {
    --- End diff --
    
    should move keys!=null out of while (add an if )


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45100757
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -390,53 +444,98 @@
           [(.getNodeId slot) (.getPort slot)]
           )))
     
    +(defn- get-version-for-key [key nimbus-host-port-info conf]
    +  (let [version (KeyVersion. key nimbus-host-port-info)]
    +    (.getKeyVersion version conf)))
    +
    +(defn get-key-seq-from-blob-store [blob-store]
    +  (let [key-iter (.listKeys blob-store nimbus-subject)]
    +    (iterator-seq key-iter)))
    +
     (defn- setup-storm-code [nimbus conf storm-id tmp-jar-location storm-conf topology]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -   (log-message "nimbus file location:" stormroot)
    -   (FileUtils/forceMkdir (File. stormroot))
    -   (FileUtils/cleanDirectory (File. stormroot))
    -   (setup-jar conf tmp-jar-location stormroot)
    -   (FileUtils/writeByteArrayToFile (File. (master-stormcode-path stormroot)) (Utils/serialize topology))
    -   (FileUtils/writeByteArrayToFile (File. (master-stormconf-path stormroot)) (Utils/toCompressedJsonConf storm-conf))
    -   (if (:code-distributor nimbus) (.upload (:code-distributor nimbus) stormroot storm-id))
    -   ))
    +  (let [subject (get-subject)
    +        storm-cluster-state (:storm-cluster-state nimbus)
    +        blob-store (:blob-store nimbus)
    +        jar-key (master-stormjar-key storm-id)
    +        code-key (master-stormcode-key storm-id)
    +        conf-key (master-stormconf-key storm-id)
    +        nimbus-host-port-info (:nimbus-host-port-info nimbus)]
    +    (when tmp-jar-location ;;in local mode there is no jar
    +      (.createBlob blob-store jar-key (FileInputStream. tmp-jar-location) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +      (if (instance? LocalFsBlobStore blob-store)
    +        (.setup-blobstore! storm-cluster-state jar-key nimbus-host-port-info (get-version-for-key jar-key nimbus-host-port-info conf))))
    +    (.createBlob blob-store conf-key (Utils/toCompressedJsonConf storm-conf) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state conf-key nimbus-host-port-info (get-version-for-key conf-key nimbus-host-port-info conf)))
    +    (.createBlob blob-store code-key (Utils/serialize topology) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state code-key nimbus-host-port-info (get-version-for-key code-key nimbus-host-port-info conf)))))
    +
    +(defn- read-storm-topology [storm-id blob-store]
    +  (Utils/deserialize
    +    (.readBlob blob-store (master-stormcode-key storm-id) (get-subject)) StormTopology))
    +
    +(defn- get-blob-replication-count [blob-key nimbus]
    +  (if (:blob-store nimbus)
    +          (-> (:blob-store nimbus)
    +            (.getBlobReplication  blob-key nimbus-subject))))
    --- End diff --
    
    Fix indentation here. The threading macro should be indented two spaces in from the `if` on the previous line, and the call to `getBlobReplication` should be indented the same amount as `(:blob-store nimbus)`.
    
    
    Also, here and elsewhere, when we are adding or changing clojure functions, we need to put the argument list on its own line in order to follow idiomatic clojure style.  There was a change we merged in a while ago that fixed many of these instances, and we should try to maintain the same style going forward.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43770299
  
    --- Diff: storm-core/src/clj/backtype/storm/cluster.clj ---
    @@ -593,18 +612,22 @@
             (let [thrift-assignment (thriftify-assignment info)]
               (set-data cluster-state (assignment-path storm-id) (Utils/serialize thrift-assignment) acls)))
     
    -      (setup-code-distributor!
    -        [this storm-id nimbusInfo]
    -        (let [path (str (code-distributor-path storm-id) "/" (.toHostPortString nimbusInfo))]
    -        (mkdirs cluster-state (code-distributor-path storm-id) acls)
    -        ;we delete the node first to ensure the node gets created as part of this session only.
    -        (delete-node cluster-state path)
    -        (set-ephemeral-node cluster-state path nil acls)))
    +      (setup-blobstore!
    +        [this key nimbusInfo versionInfo]
    +        (let [path (str (blobstore-path key) "/" (.toHostPortString nimbusInfo) "-" versionInfo)]
    +          (mkdirs cluster-state (blobstore-path key) acls)
    +          ;we delete the node first to ensure the node gets created as part of this session only.
    +          (delete-node-blobstore cluster-state (str (blobstore-path key)) (.toHostPortString nimbusInfo))
    +          (set-ephemeral-node cluster-state path nil acls)))
    +
    +      (remove-blobstore-key!
    +        [this blob-key]
    +        (log-debug "removing key" blob-key)
    +        (delete-node cluster-state (blobstore-path blob-key)))
     
           (remove-storm!
             [this storm-id]
             (delete-node cluster-state (assignment-path storm-id))
    -        (delete-node cluster-state (code-distributor-path storm-id))
    --- End diff --
    
    How are the storm-jar, storm-conf, and storm-topology deleted from the blob store when the topology is removed?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46094594
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStoreUtils.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.security.auth.NimbusPrincipal;
    +import backtype.storm.utils.NimbusClient;
    +import backtype.storm.utils.Utils;
    +import backtype.storm.utils.ZookeeperAuthInfo;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.thrift.transport.TTransportException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class BlobStoreUtils {
    +    private static final String BLOBSTORE_SUBTREE="/blobstore";
    +    private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
    +
    +    public static CuratorFramework createZKClient(Map conf) {
    +        List<String> zkServers = (List<String>) conf.get(Config.STORM_ZOOKEEPER_SERVERS);
    +        Object port = conf.get(Config.STORM_ZOOKEEPER_PORT);
    +        ZookeeperAuthInfo zkAuthInfo = new ZookeeperAuthInfo(conf);
    +        CuratorFramework zkClient = Utils.newCurator(conf, zkServers, port, (String) conf.get(Config.STORM_ZOOKEEPER_ROOT), zkAuthInfo);
    +        zkClient.start();
    +        return zkClient;
    +    }
    +
    +    public static Subject getNimbusSubject() {
    +        Subject subject = new Subject();
    +        subject.getPrincipals().add(new NimbusPrincipal());
    +        return subject;
    +    }
    +
    +    // Normalize state
    +    public static BlobKeySequenceInfo normalizeNimbusHostPortSequenceNumberInfo(String nimbusSeqNumberInfo) {
    +        BlobKeySequenceInfo keySequenceInfo = new BlobKeySequenceInfo();
    +        int lastIndex = nimbusSeqNumberInfo.lastIndexOf("-");
    +        keySequenceInfo.setNimbusHostPort(nimbusSeqNumberInfo.substring(0, lastIndex));
    +        keySequenceInfo.setSequenceNumber(nimbusSeqNumberInfo.substring(lastIndex + 1));
    +        return keySequenceInfo;
    +    }
    +
    +    // Check for latest sequence number of a key inside zookeeper and return nimbodes containing the latest sequence number
    +    public static Set<NimbusInfo> getNimbodesWithLatestSequenceNumberOfBlob(CuratorFramework zkClient, String key) throws Exception {
    +        List<String> stateInfoList = zkClient.getChildren().forPath("/blobstore/" + key);
    +        Set<NimbusInfo> nimbusInfoSet = new HashSet<NimbusInfo>();
    +        int latestSeqNumber = getLatestSequenceNumber(stateInfoList);
    +        LOG.debug("getNimbodesWithLatestSequenceNumberOfBlob stateInfo {} version {}", stateInfoList, latestSeqNumber);
    +        // Get the nimbodes with the latest version
    +        for(String state : stateInfoList) {
    +            BlobKeySequenceInfo sequenceInfo = normalizeNimbusHostPortSequenceNumberInfo(state);
    +            if (latestSeqNumber == Integer.parseInt(sequenceInfo.getSequenceNumber())) {
    +                nimbusInfoSet.add(NimbusInfo.parse(sequenceInfo.getNimbusHostPort()));
    +            }
    +        }
    +        LOG.debug("nimbusInfoList {}", nimbusInfoSet);
    +        return nimbusInfoSet;
    +    }
    +
    +    // Get sequence number details from latest sequence number of the blob
    +    public static int getLatestSequenceNumber(List<String> stateInfoList) {
    +        int seqNumber = 0;
    +        // Get latest sequence number of the blob present in the zookeeper --> possible to refactor this piece of code
    +        for (String state : stateInfoList) {
    +            BlobKeySequenceInfo sequenceInfo = normalizeNimbusHostPortSequenceNumberInfo(state);
    +            if (seqNumber < Integer.parseInt(sequenceInfo.getSequenceNumber())) {
    +                seqNumber = Integer.parseInt(sequenceInfo.getSequenceNumber());
    +                LOG.debug("Sequence Info {}", seqNumber);
    +            }
    +        }
    +        LOG.debug("Latest Sequence Number {}", seqNumber);
    +        return seqNumber;
    +    }
    +
    +    // Download missing blobs from potential nimbodes
    +    public static boolean downloadMissingBlob(Map conf, BlobStore blobStore, String key, Set<NimbusInfo> nimbusInfos)
    +            throws TTransportException {
    +        NimbusClient client = null;
    +        ReadableBlobMeta rbm = null;
    +        ClientBlobStore remoteBlobStore = null;
    +        InputStreamWithMeta in = null;
    +        Boolean isSuccess = false;
    +        LOG.debug("Download blob NimbusInfos {}", nimbusInfos);
    +        for (NimbusInfo nimbusInfo : nimbusInfos) {
    +            if(isSuccess) {
    +                break;
    +            }
    +            try {
    +                client = new NimbusClient(conf, nimbusInfo.getHost(), nimbusInfo.getPort(), null);
    +                rbm = client.getClient().getBlobMeta(key);
    +                remoteBlobStore = new NimbusBlobStore();
    +                remoteBlobStore.setClient(conf, client);
    +                in = remoteBlobStore.getBlob(key);
    +                blobStore.createBlob(key, in, rbm.get_settable(), getNimbusSubject());
    +                // if key already exists while creating the blob else update it
    +                Iterator<String> keyIterator = blobStore.listKeys();
    +                while (keyIterator.hasNext()) {
    +                    if (keyIterator.next().equals(key)) {
    +                        LOG.debug("Success creating key, {}", key);
    +                        isSuccess = true;
    +                        break;
    +                    }
    +                }
    +            } catch (IOException | AuthorizationException exception) {
    +                throw new RuntimeException(exception);
    +            } catch (KeyAlreadyExistsException kae) {
    +                LOG.info("KeyAlreadyExistsException Key: {} {}", key, kae);
    +            } catch (KeyNotFoundException knf) {
    +                // Catching and logging KeyNotFoundException because, if
    +                // there is a subsequent update and delete, the non-leader
    +                // nimbodes might throw an exception.
    +                LOG.info("KeyNotFoundException Key: {} {}", key, knf);
    +            } catch (Exception exp) {
    +                // Logging an exception while client is connecting
    +                LOG.error("Exception {}", exp);
    +            }
    +        }
    +
    +        if (!isSuccess) {
    +            LOG.error("Could not download blob with key" + key);
    +            return false;
    +        }
    +        return isSuccess;
    +    }
    +
    +    // Download updated blobs from potential nimbodes
    +    public static boolean downloadUpdatedBlob(Map conf, BlobStore blobStore, String key, Set<NimbusInfo> nimbusInfos)
    +            throws TTransportException {
    +        NimbusClient client = null;
    +        ClientBlobStore remoteBlobStore = null;
    +        InputStreamWithMeta in = null;
    +        AtomicOutputStream out = null;
    +        Boolean isSuccess = false;
    +        LOG.debug("Download blob NimbusInfos {}", nimbusInfos);
    +        for (NimbusInfo nimbusInfo : nimbusInfos) {
    +            if (isSuccess) {
    +                break;
    +            }
    +            try {
    +                client = new NimbusClient(conf, nimbusInfo.getHost(), nimbusInfo.getPort(), null);
    +                remoteBlobStore = new NimbusBlobStore();
    +                remoteBlobStore.setClient(conf, client);
    +                in = remoteBlobStore.getBlob(key);
    +                out = blobStore.updateBlob(key, getNimbusSubject());
    +                byte[] buffer = new byte[2048];
    +                int len = 0;
    +                while ((len = in.read(buffer)) > 0) {
    +                    out.write(buffer, 0, len);
    +                }
    +                if (out != null) {
    +                    out.close();
    +                }
    +                isSuccess = true;
    +            } catch (IOException | AuthorizationException exception) {
    +                throw new RuntimeException(exception);
    +            } catch (KeyNotFoundException knf) {
    +                // Catching and logging KeyNotFoundException because, if
    +                // there is a subsequent update and delete, the non-leader
    +                // nimbodes might throw an exception.
    +                LOG.info("KeyNotFoundException {}", knf);
    +            } catch (Exception exp) {
    +                // Logging an exception while client is connecting
    +                LOG.error("Exception {}", exp);
    +            }
    +        }
    +
    +        if (!isSuccess) {
    +            LOG.error("Could not update the blob with key" + key);
    +            return false;
    +        }
    +        return isSuccess;
    +    }
    +
    +    // Get the list of keys from blobstore
    +    public static List<String> getKeyListFromBlobStore(BlobStore blobStore) throws Exception {
    +        Iterator<String> keys = blobStore.listKeys();
    +        List<String> keyList = new ArrayList<String>();
    +        while (keys != null && keys.hasNext()) {
    --- End diff --
    
    I am right, as this is a short circuit operator it will not execute if keys != null. I do not think it will make any difference if I move this out. Might suggest if I am wrong


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r44997547
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -732,25 +863,89 @@
      (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
       (worker-launcher-and-wait conf (storm-conf TOPOLOGY-SUBMITTER-USER) ["code-dir" dir] :log-prefix (str "setup conf for " dir))))
     
    +(defn setup-blob-permission [conf storm-conf path]
    +  (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
    +    (worker-launcher-and-wait conf (storm-conf TOPOLOGY-SUBMITTER-USER) ["blob" path] :log-prefix (str "setup blob permissions for " path))))
    +
    +(defn setup-storm-code-dir [conf storm-conf dir]
    +  (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
    +    (worker-launcher-and-wait conf (storm-conf TOPOLOGY-SUBMITTER-USER) ["code-dir" dir] :log-prefix (str "setup conf for " dir))))
    +
    +(defn download-blobs-for-topology!
    +  "Download all blobs listed in the topology configuration for a given topology."
    +  [conf stormconf-path localizer tmproot]
    +  (let [storm-conf (read-supervisor-storm-conf-given-path conf stormconf-path)
    +        blobstore-map (storm-conf TOPOLOGY-BLOBSTORE-MAP)
    +        user (storm-conf TOPOLOGY-SUBMITTER-USER)
    +        topo-name (storm-conf TOPOLOGY-NAME)
    +        user-dir (.getLocalUserFileCacheDir localizer user)
    +        localresources (blobstore-map-to-localresources blobstore-map)]
    +    (when localresources
    +      (when-not (.exists user-dir)
    +        (FileUtils/forceMkdir user-dir)
    +        (setup-blob-permission conf storm-conf (.toString user-dir)))
    +      (try
    +        (let [localized-resources (.getBlobs localizer localresources user topo-name user-dir)]
    +          (setup-blob-permission conf storm-conf (.toString user-dir))
    +          (doseq [local-rsrc localized-resources]
    +            (let [rsrc-file-path (File. (.getFilePath local-rsrc))
    +                  key-name (.getName rsrc-file-path)
    +                  blob-symlink-target-name (.getName (File. (.getCurrentSymlinkPath local-rsrc)))
    +                  symlink-name (get-blob-localname (get blobstore-map key-name) key-name)]
    +              (create-symlink! tmproot (.getParent rsrc-file-path) symlink-name
    +                blob-symlink-target-name))))
    +        (catch AuthorizationException authExp
    +          (log-error authExp))
    +        (catch KeyNotFoundException knf
    +          (log-error knf))))))
    +
    +(defn get-blob-file-names
    +  [blobstore-map]
    +  (if blobstore-map
    +    (for [[k, data] blobstore-map] (get-blob-localname data k))))
    --- End diff --
    
    body on own line.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43788654
  
    --- Diff: bin/storm.py ---
    @@ -263,6 +263,34 @@ def upload_credentials(*args):
             jvmtype="-client",
             extrajars=[USER_CONF_DIR, STORM_BIN_DIR])
     
    +def blobstore(*args):
    +    """Syntax: [storm blobstore [cmd]]
    +
    +    list [KEY...] - lists blobs currently in the blob store
    +    cat [-f FILE] KEY - read a blob and then either write it to a file, or STDOUT (requires read access).
    +    create [-f FILE] [-a ACL ...] [--repl-fctr NUMBER] KEY - create a new blob. Contents comes from a FILE
    +         or STDIN. ACL is in the form [uo]:[username]:[r-][w-][a-] can be comma
    +         separated list.
    +         --repl-fctr refers to replication factor for the blob. Here NUMBER > 0.
    +         for example the following would create 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.
    +         storm blobstore create mytopo:data.tgz -f data.tgz -a u:alice:rwa,u:bob:rw,o::r
    --- End diff --
    
    I didn't read down far enough.  Maybe just move this to a general ACL example below, and some brief pointer to the example from the other commands that take ACL option arguments?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43952116
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -984,25 +1108,30 @@
             ))))
     
     (defn cleanup-corrupt-topologies! [nimbus]
    -  (if (is-leader nimbus :throw-exception false)
    -    (let [storm-cluster-state (:storm-cluster-state nimbus)
    -          code-ids (set (code-ids (:conf nimbus)))
    -          active-topologies (set (.active-storms storm-cluster-state))
    -          corrupt-topologies (set/difference active-topologies code-ids)]
    -      (doseq [corrupt corrupt-topologies]
    -        (log-message "Corrupt topology " corrupt " has state on zookeeper but doesn't have a local dir on Nimbus. Cleaning up...")
    -        (.remove-storm! storm-cluster-state corrupt)
    -        )))
    -  (log-message "not a leader, skipping cleanup-corrupt-topologies"))
    -
    -;;setsup code distributor entries for all current topologies for which code is available locally.
    -(defn setup-code-distributor [nimbus]
       (let [storm-cluster-state (:storm-cluster-state nimbus)
    -        locally-available-storm-ids (set (code-ids (:conf nimbus)))
    +        blob-store (:blob-store nimbus)
    +        code-ids (set (code-ids blob-store))
             active-topologies (set (.active-storms storm-cluster-state))
    -        locally-available-active-storm-ids (set/intersection locally-available-storm-ids active-topologies)]
    -    (doseq [storm-id locally-available-active-storm-ids]
    -      (.setup-code-distributor! storm-cluster-state storm-id (:nimbus-host-port-info nimbus)))))
    +        corrupt-topologies (set/difference active-topologies code-ids)]
    +    (doseq [corrupt corrupt-topologies]
    +      (log-message "Corrupt topology " corrupt " has state on zookeeper but doesn't have a local dir on Nimbus. Cleaning up...")
    +      (.remove-storm! storm-cluster-state corrupt)
    +      (if (instance? LocalFsBlobStore blob-store)
    +        (doseq [blob-key (get-key-list-from-id (:conf nimbus) corrupt)]
    +          (.remove-blobstore-key! storm-cluster-state blob-key)))
    +      )))
    +
    +;;setsup blobstore for all current keys
    +(defn setup-blobstore [nimbus]
    +  (let [storm-cluster-state (:storm-cluster-state nimbus)
    +        blob-store (:blob-store nimbus)
    +        local-set-of-keys (get-key-set-from-blob-store blob-store)
    +        all-keys (set (.active-keys storm-cluster-state))
    +        locally-available-active-keys (set/intersection local-set-of-keys all-keys)]
    +    (log-debug "Creating list of key entries for blobstore inside zookeeper" all-keys "local" locally-available-active-keys)
    +    (doseq [key locally-available-active-keys]
    +      (.setup-blobstore! storm-cluster-state key (:nimbus-host-port-info nimbus) (get-metadata-version blob-store key get-nimbus-subject))
    +      )))
    --- End diff --
    
    dangling parens


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45131380
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -326,16 +331,28 @@
                                              (log-error t "Error when processing event")
                                              (exit-process! 20 "Error when processing an event")
                                              ))
    +   :blob-update-timer (mk-timer :kill-fn (fn [t]
    +                                           (log-error t "Error when processing event")
    +                                           (exit-process! 20 "Error when processing a event")))
    +   :localizer (Utils/createLocalizer conf (supervisor-local-dir conf))
        :assignment-versions (atom {})
        :sync-retry (atom 0)
    -   :code-distributor (mk-code-distributor conf)
        :download-lock (Object.)
        :stormid->profiler-actions (atom {})
        })
     
    +(defn required-topo-files-exist?
    +  [conf storm-id]
    +  (let [stormroot (supervisor-stormdist-root conf storm-id)
    +        stormjarpath (supervisor-stormjar-path stormroot)
    +        stormcodepath (supervisor-stormcode-path stormroot)
    +        stormconfpath (supervisor-stormconf-path stormroot)]
    +    (and (every? exists-file? [stormroot stormconfpath stormcodepath])
    +    (or (local-mode? conf)
    +      (exists-file? stormjarpath)))))
    --- End diff --
    
    * Indent `or` with `every?`
    * Indent `exists-file?` with `local-mode?`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43937682
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -130,22 +162,40 @@
     (defn inbox [nimbus]
       (master-inbox (:conf nimbus)))
     
    -(defn- read-storm-conf [conf storm-id]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -    (merge conf
    -       (clojurify-structure
    -         (Utils/fromCompressedJsonConf
    -           (FileUtils/readFileToByteArray
    -             (File. (master-stormconf-path stormroot))))))))
    +(defn- get-subject []
    +  (let [req (ReqContext/context)]
    +    (.subject req)))
    +
    +(defn- read-storm-conf [conf storm-id blob-store]
    +  (clojurify-structure
    +    (Utils/fromCompressedJsonConf
    +      (.readBlob blob-store (master-stormconf-key storm-id) (get-subject)))))
     
     (declare delay-event)
     (declare mk-assignments)
     
    +(defn nimbus-subject
    +  []
    +  (let [subject (Subject.)
    +        principal (NimbusPrincipal.)
    +        principals (.getPrincipals subject)]
    +    (.add principals principal)
    +    subject))
    +
    +(def get-nimbus-subject
    +  (nimbus-subject))
    --- End diff --
    
    Looks like the var `get-nimbus-subject` and the function above `nimbus-subject` it have swapped names.  Can we switch them?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43797208
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -346,53 +394,124 @@
           [(.getNodeId slot) (.getPort slot)]
           )))
     
    +(defn- get-nimbus-subject []
    +  (let [nimbus-subject (Subject.)
    +        nimbus-principal (NimbusPrincipal.)
    +        principals (.getPrincipals nimbus-subject)]
    +    (.add principals nimbus-principal)
    +    nimbus-subject))
    +
    +(defn- get-metadata-version [blob-store key subject]
    +  (let [blob-meta (.getBlobMeta blob-store key subject)]
    +    (.get_version blob-meta)))
    +
    +(defn get-key-list-from-blob-store [blob-store]
    +  (let [key-iter (.listKeys blob-store (get-nimbus-subject))
    +        keys (iterator-seq key-iter)]
    +    (if (not-nil? keys)
    +      (java.util.ArrayList. keys)
    +      [])))
    +
     (defn- setup-storm-code [nimbus conf storm-id tmp-jar-location storm-conf topology]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -   (log-message "nimbus file location:" stormroot)
    -   (FileUtils/forceMkdir (File. stormroot))
    -   (FileUtils/cleanDirectory (File. stormroot))
    -   (setup-jar conf tmp-jar-location stormroot)
    -   (FileUtils/writeByteArrayToFile (File. (master-stormcode-path stormroot)) (Utils/serialize topology))
    -   (FileUtils/writeByteArrayToFile (File. (master-stormconf-path stormroot)) (Utils/toCompressedJsonConf storm-conf))
    -   (if (:code-distributor nimbus) (.upload (:code-distributor nimbus) stormroot storm-id))
    -   ))
    +  (let [subject (get-subject)
    +        storm-cluster-state (:storm-cluster-state nimbus)
    +        blob-store (:blob-store nimbus)
    +        jar-key (master-stormjar-key storm-id)
    +        code-key (master-stormcode-key storm-id)
    +        conf-key (master-stormconf-key storm-id)
    +        nimbus-host-port-info (:nimbus-host-port-info nimbus)]
    +    (log-message "subject-changed" subject)
    +    (if tmp-jar-location ;;in local mode there is no jar
    +      (do
    +        (log-message "tmp-jar-location" tmp-jar-location)
    +        (.createBlob blob-store jar-key (FileInputStream. tmp-jar-location) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +        (if (instance? LocalFsBlobStore blob-store)
    +          (.setup-blobstore! storm-cluster-state jar-key nimbus-host-port-info (get-metadata-version blob-store jar-key subject)))
    +        ))
    --- End diff --
    
    danglers


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43814238
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStore.java ---
    @@ -0,0 +1,264 @@
    +/**
    + * 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.daemon.Shutdownable;
    +import backtype.storm.generated.*;
    +import backtype.storm.nimbus.NimbusInfo;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.util.*;
    --- End diff --
    
    And here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43812156
  
    --- Diff: storm-core/src/clj/backtype/storm/cluster.clj ---
    @@ -436,18 +445,27 @@
     
             (set-ephemeral-node cluster-state (nimbus-path nimbus-id) (Utils/serialize nimbus-summary) acls))
     
    -      (code-distributor-info
    -        [this storm-id]
    -        (map (fn [nimbus-info] (NimbusInfo/parse nimbus-info))
    -          (let [path (code-distributor-path storm-id)]
    -            (do
    -              (sync-path cluster-state path)
    -              (get-children cluster-state path false)))))
    +      (blobstore-key-details
    +        [this]
    +        (do
    +          (sync-path cluster-state BLOBSTORE-SUBTREE)
    +          (get-children cluster-state BLOBSTORE-SUBTREE false)))
    +
    +      (blobstore-info
    +        [this blob-key]
    +        (let [path (blobstore-path blob-key)]
    +          (do
    +            (sync-path cluster-state path)
    +            (get-children cluster-state path false))))
    --- End diff --
    
    Remove `do`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45793192
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/KeySequenceNumber.java ---
    @@ -0,0 +1,227 @@
    +/**
    + * 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.zookeeper.CreateMode;
    +import org.apache.zookeeper.ZooDefs;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.nio.ByteBuffer;
    +import java.util.TreeSet;
    +import java.util.Map;
    +import java.util.List;
    +
    +/**
    + * Class hands over the key sequence number which implies the number of updates made to a blob.
    + * The information regarding the keys and the sequence number which represents the number of updates are
    + * stored within the zookeeper in the following format.
    + * /storm/blobstore/key_name/nimbushostport-sequencenumber
    + * Example:
    + * If there are two nimbodes with nimbus.seeds:leader,non-leader are set,
    + * then the state inside the zookeeper is eventually stored as:
    + * /storm/blobstore/key1/leader:8080-1
    + * /storm/blobstore/key1/non-leader:8080-1
    + * indicates that a new blob with the name key1 has been created on the leader
    + * nimbus and the non-leader nimbus syncs after a call back is triggered by attempting
    + * to download the blob and finally updates its state inside the zookeeper.
    + *
    + * A watch is placed on the /storm/blobstore/key1 and the znodes leader:8080-1 and
    + * non-leader:8080-1 are ephemeral which implies that these nodes exist only until the
    + * connection between the corresponding nimbus and the zookeeper persist. If in case the
    + * nimbus crashes the node disappears under /storm/blobstore/key1.
    + *
    + * The sequence number for the keys are handed over based on the following scenario:
    + * Lets assume there are three nimbodes up and running, one being the leader and the other
    + * being the non-leader.
    + *
    + * 1. Create is straight forward.
    + * Check whether the znode -> /storm/blobstore/key1 has been created or not. It implies
    + * the blob has not been created yet. If not created, it creates it and updates the zookeeper
    + * states under /storm/blobstore/key1 and /storm/blobstoremaxkeysequencenumber/key1.
    + * The znodes it creates on these nodes are /storm/blobstore/key1/leader:8080-1,
    + * /storm/blobstore/key1/non-leader:8080-1 and /storm/blobstoremaxkeysequencenumber/key1/1.
    + * The later holds the global sequence number across all nimbodes more like a static variable
    + * indicating the true value of number of updates for a blob. This node helps to maintain sanity in case
    + * leadership changes due to crashing.
    + *
    + * 2. Delete does not require to hand over the sequence number.
    + *
    + * 3. Finally, the update has few scenarios.
    + *
    + *  The class implements a TreeSet. The basic idea is if all the nimbodes have the same
    + *  sequence number for the blob, then the number of elements in the set is 1 which holds
    + *  the latest value of sequence number. If the number of elements are greater than 1 then it
    + *  implies that there is sequence mismatch and there is need for syncing the blobs across
    + *  nimbodes.
    + *
    + *  The logic for handing over sequence numbers based on the state are described as follows
    + *  Here consider Nimbus-1 alias as N1 and Nimbus-2 alias as N2.
    + *  Scenario 1:
    + *  Example: Normal create/update scenario
    + *  Operation     Nimbus-1:state     Nimbus-2:state     Seq-Num-Nimbus-1  Seq-Num-Nimbus-2          Max-Seq-Num
    + *  Create-Key1   alive - Leader     alive              1                                           1
    + *  Sync          alive - Leader     alive              1                 1 (callback -> download)  1
    + *  Update-Key1   alive - Leader     alive              2                 1                         2
    + *  Sync          alive - Leader     alive              2                 2 (callback -> download)  2
    + *
    + *  Scenario 2:
    + *  Example: Leader nimbus crash followed by leader election, update and ex-leader restored again
    + *  Operation     Nimbus-1:state     Nimbus-2:state     Seq-Num-Nimbus-1  Seq-Num-Nimbus-2          Max-Seq-Num
    + *  Create        alive - Leader     alive              1                                           1
    + *  Sync          alive - Leader     alive              1                 1 (callback -> download)  1
    + *  Update        alive - Leader     alive              2                 1                         2
    + *  Sync          alive - Leader     alive              2                 2 (callback -> download)  2
    + *  Update        alive - Leader     alive              3                 2                         3
    + *  Crash         crash - Leader     alive              3                 2                         3
    + *  New - Leader  crash              alive - Leader     3 (Invalid)       2                         3
    + *  Update        crash              alive - Leader     3 (Invalid)       4 (max-seq-num + 1)       4
    + *  N1-Restored   alive              alive - Leader     0                 4                         4
    + *  Sync          alive              alive - Leader     4                 4                         4
    + *
    + *  Scenario 3:
    + *  Example: Leader nimbus crash followed by leader election, update and ex-leader restored again
    + *  Operation     Nimbus-1:state     Nimbus-2:state     Seq-Num-Nimbus-1  Seq-Num-Nimbus-2          Max-Seq-Num
    + *  Create        alive - Leader     alive              1                                           1
    + *  Sync          alive - Leader     alive              1                 1 (callback -> download)  1
    + *  Update        alive - Leader     alive              2                 1                         2
    + *  Sync          alive - Leader     alive              2                 2 (callback -> download)  2
    + *  Update        alive - Leader     alive              3                 2                         3
    + *  Crash         crash - Leader     alive              3                 2                         3
    + *  Elect Leader  crash              alive - Leader     3 (Invalid)       2                         3
    + *  N1-Restored   alive              alive - Leader     3                 2                         3
    + *  Read/Update   alive              alive - Leader     3                 4 (Downloads from N1)     4
    + *  Sync          alive              alive - Leader     4 (callback)      4                         4
    + *  Here the download is triggered whenever an operation corresponding to the blob is triggered on the
    + *  nimbus like a read or update operation. Here, in the read/update call it is hard to know which call
    + *  is read or update. Hence, by incrementing the sequence number to max-seq-num + 1 we ensure that the
    + *  synchronization happens appropriately and all nimbodes have the same blob.
    + */
    +public class KeySequenceNumber {
    +    private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
    +    private final String BLOBSTORE_SUBTREE="/blobstore";
    +    private final String BLOBSTORE_MAX_KEY_SEQUENCE_SUBTREE="/blobstoremaxkeysequencenumber";
    +    private final String key;
    +    private final NimbusInfo nimbusInfo;
    +    private final int INT_CAPACITY = 4;
    +
    +    public KeySequenceNumber(String key, NimbusInfo nimbusInfo) {
    +        this.key = key;
    +        this.nimbusInfo = nimbusInfo;
    +    }
    +
    +    public int getKeySequenceNumber(Map conf) {
    +        TreeSet<Integer> sequenceNumbers = new TreeSet<Integer>();
    +        CuratorFramework zkClient = BlobStoreUtils.createZKClient(conf);
    +        try {
    +            // Key has not been created yet and it is the first time it is being created
    +            if(zkClient.checkExists().forPath(BLOBSTORE_SUBTREE + "/" + key) == null) {
    +                zkClient.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT)
    +                        .withACL(ZooDefs.Ids.OPEN_ACL_UNSAFE).forPath(BLOBSTORE_MAX_KEY_SEQUENCE_SUBTREE + "/" + key);
    +                zkClient.setData().forPath(BLOBSTORE_MAX_KEY_SEQUENCE_SUBTREE + "/" + key,
    +                        ByteBuffer.allocate(INT_CAPACITY).putInt(1).array());
    +                return 1;
    +            }
    +
    +            // When all nimbodes go down and one or few of them come up
    +            // Unfortunately there might not be an exact way to know which one contains the most updated blob,
    +            // if all go down which is unlikely. Hence there might be a need to update the blob if all go down.
    +            List<String> stateInfoList = zkClient.getChildren().forPath(BLOBSTORE_SUBTREE + "/" + key);
    +            LOG.debug("stateInfoList-size {} stateInfoList-data {}", stateInfoList.size(), stateInfoList);
    +            if(stateInfoList.isEmpty()) {
    +                return getMaxSequenceNumber(zkClient);
    +            }
    +
    +            LOG.debug("stateInfoSize {}", stateInfoList.size());
    +            // In all other cases check for the latest update sequence of the blob on the nimbus
    +            // and assign the appropriate number. Check if all are have same sequence number,
    +            // if not assign the highest sequence number.
    +            for (String stateInfo:stateInfoList) {
    +                sequenceNumbers.add(Integer.parseInt(BlobStoreUtils.normalizeVersionInfo(stateInfo)[1]));
    +            }
    +
    +            // Update scenario 2 and 3 explain the code logic written here
    +            // especially when nimbus crashes and comes up after and before update
    +            // respectively.
    +            int currentSeqNumber = getMaxSequenceNumber(zkClient);
    +            if (!checkIfStateContainsCurrentNimbusHost(stateInfoList, nimbusInfo) && !nimbusInfo.isLeader()) {
    +                if (sequenceNumbers.last() < currentSeqNumber) {
    +                    return currentSeqNumber;
    +                } else {
    +                    return 0;
    +                }
    +            }
    +
    +            // It covers scenario's expalined in scenario 3 when nimbus-1 holding the latest
    --- End diff --
    
    `scenarios`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r44998736
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -326,16 +331,29 @@
                                              (log-error t "Error when processing event")
                                              (exit-process! 20 "Error when processing an event")
                                              ))
    +   :blob-update-timer (mk-timer :kill-fn (fn [t]
    +                                           (log-error t "Error when processing blob-update")
    +                                           (exit-process! 20 "Error when processing a blob-update")
    --- End diff --
    
    it would make it more clear but i guess in order to keep it consistent with other's we might want to change it


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by ptgoetz <gi...@git.apache.org>.
Github user ptgoetz commented on the pull request:

    https://github.com/apache/storm/pull/845#issuecomment-153419952
  
    Based on the associated JIRA and from looking at the commits this pull request contains, this needs to go through the IP Clearance process [1].
    
    From what I can tell, it's a significant amount of code that was developed internally at Yahoo, possibly by more than one contributor. The first commit introduces a lot of the code, and on the surface it's unclear where it came from.
    
    The process is easy, we would just need a grant from Yahoo, and a list of contributors. Then it's just a matter of filling out the paperwork and waiting 72 hrs. for review.
    
    I'll volunteer to do all the IP Clearance work/coordination.
    
    [1] http://incubator.apache.org/ip-clearance/index.html


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket closed the pull request at:

    https://github.com/apache/storm/pull/845


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43956807
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -346,53 +400,104 @@
           [(.getNodeId slot) (.getPort slot)]
           )))
     
    +(defn- get-metadata-version [blob-store key subject]
    +  (let [blob-meta (.getBlobMeta blob-store key subject)]
    +    (.get_version blob-meta)))
    +
    +(defn get-key-set-from-blob-store [blob-store]
    +  (let [key-iter (.listKeys blob-store get-nimbus-subject)]
    +    (set (iterator-seq key-iter))))
    +
     (defn- setup-storm-code [nimbus conf storm-id tmp-jar-location storm-conf topology]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -   (log-message "nimbus file location:" stormroot)
    -   (FileUtils/forceMkdir (File. stormroot))
    -   (FileUtils/cleanDirectory (File. stormroot))
    -   (setup-jar conf tmp-jar-location stormroot)
    -   (FileUtils/writeByteArrayToFile (File. (master-stormcode-path stormroot)) (Utils/serialize topology))
    -   (FileUtils/writeByteArrayToFile (File. (master-stormconf-path stormroot)) (Utils/toCompressedJsonConf storm-conf))
    -   (if (:code-distributor nimbus) (.upload (:code-distributor nimbus) stormroot storm-id))
    -   ))
    +  (let [subject (get-subject)
    +        storm-cluster-state (:storm-cluster-state nimbus)
    +        blob-store (:blob-store nimbus)
    +        jar-key (master-stormjar-key storm-id)
    +        code-key (master-stormcode-key storm-id)
    +        conf-key (master-stormconf-key storm-id)
    +        nimbus-host-port-info (:nimbus-host-port-info nimbus)]
    +    (if tmp-jar-location ;;in local mode there is no jar
    +      (do
    +        (.createBlob blob-store jar-key (FileInputStream. tmp-jar-location) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +        (if (instance? LocalFsBlobStore blob-store)
    +          (.setup-blobstore! storm-cluster-state jar-key nimbus-host-port-info (get-metadata-version blob-store jar-key subject)))))
    +    (.createBlob blob-store conf-key (Utils/toCompressedJsonConf storm-conf) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state conf-key nimbus-host-port-info (get-metadata-version blob-store conf-key subject)))
    +    (.createBlob blob-store code-key (Utils/serialize topology) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state code-key nimbus-host-port-info (get-metadata-version blob-store code-key subject)))))
    +
    +(defn- read-storm-topology [storm-id blob-store]
    +  (Utils/deserialize
    +    (.readBlob blob-store (master-stormcode-key storm-id) (get-subject)) StormTopology))
    +
    +(defn- get-blob-replication-count [blob-key nimbus]
    +  (if (:blob-store nimbus)
    +          (-> (:blob-store nimbus)
    +            (.getBlobReplication  blob-key get-nimbus-subject)
    +            (.get_replication))))
     
     (defn- wait-for-desired-code-replication [nimbus conf storm-id]
       (let [min-replication-count (conf TOPOLOGY-MIN-REPLICATION-COUNT)
             max-replication-wait-time (conf TOPOLOGY-MAX-REPLICATION-WAIT-TIME-SEC)
    -        total-wait-time (atom 0)
    -        current-replication-count (atom (if (:code-distributor nimbus) (.getReplicationCount (:code-distributor nimbus) storm-id) 0))]
    -  (if (:code-distributor nimbus)
    -    (while (and (> min-replication-count @current-replication-count)
    -             (or (= -1 max-replication-wait-time)
    -               (< @total-wait-time max-replication-wait-time)))
    +        current-replication-count-jar (if (not (local-mode? conf)) (atom
    +                                                                     (get-blob-replication-count (master-stormjar-key storm-id) nimbus))
    +                                                                     (atom min-replication-count))
    +        current-replication-count-code (atom (get-blob-replication-count (master-stormcode-key storm-id) nimbus))
    +        current-replication-count-conf (atom (get-blob-replication-count (master-stormconf-key storm-id) nimbus))
    +        total-wait-time (atom 0)]
    +    (log-message "wait for desired replication" "count"
    +      min-replication-count "wait-time" max-replication-wait-time
    +      "code" @current-replication-count-code
    +      "conf" @current-replication-count-conf
    +      "jar" @current-replication-count-jar
    +      "replication count" (get-blob-replication-count (master-stormconf-key storm-id) nimbus))
    +    (if (:blob-store nimbus)
    +      (while (and (> min-replication-count @current-replication-count-jar)
    +               (> min-replication-count @current-replication-count-code)
    +               (> min-replication-count @current-replication-count-conf)
    +               (or (= -1 max-replication-wait-time)
    +                 (< @total-wait-time max-replication-wait-time)))
             (sleep-secs 1)
             (log-debug "waiting for desired replication to be achieved.
               min-replication-count = " min-replication-count  " max-replication-wait-time = " max-replication-wait-time
    -          "current-replication-count = " @current-replication-count " total-wait-time " @total-wait-time)
    +          (if (not (local-mode? conf))"current-replication-count for jar key = " @current-replication-count-jar)
    +          "current-replication-count for code key = " @current-replication-count-code
    +          "current-replication-count for conf key = " @current-replication-count-conf
    +          " total-wait-time " @total-wait-time)
             (swap! total-wait-time inc)
    -        (reset! current-replication-count  (.getReplicationCount (:code-distributor nimbus) storm-id))))
    -  (if (< min-replication-count @current-replication-count)
    -    (log-message "desired replication count "  min-replication-count " achieved,
    -      current-replication-count" @current-replication-count)
    -    (log-message "desired replication count of "  min-replication-count " not achieved but we have hit the max wait time "
    -      max-replication-wait-time " so moving on with replication count = " @current-replication-count)
    -    )))
    -
    -(defn- read-storm-topology [conf storm-id]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -    (Utils/deserialize
    -      (FileUtils/readFileToByteArray
    -        (File. (master-stormcode-path stormroot))
    -        ) StormTopology)))
    +        (if (not (local-mode? conf))(reset! current-replication-count-conf  (get-blob-replication-count (master-stormconf-key storm-id))))
    +          (reset! current-replication-count-code  (get-blob-replication-count (master-stormcode-key storm-id)))
    +          (reset! current-replication-count-jar  (get-blob-replication-count (master-stormjar-key storm-id)))))
    --- End diff --
    
    i am passing in two arguments, will try to make the indentation clear


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45760689
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/KeyVersion.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * 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.zookeeper.CreateMode;
    +import org.apache.zookeeper.ZooDefs;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.TreeSet;
    +import java.util.Map;
    +import java.util.List;
    +
    +/**
    + * Class hands over the version of the key to be stored within the zookeeper
    + */
    +public class KeyVersion {
    +  private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
    +  private final String BLOBSTORE_SUBTREE="/blobstore";
    +  private final String BLOBSTORE_KEY_COUNTER_SUBTREE="/blobstorekeycounter";
    +  private String key;
    +  private NimbusInfo nimbusInfo;
    --- End diff --
    
    @d2r addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43935294
  
    --- 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])
    --- End diff --
    
    Let's combine this import with the other from `java.util` below.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43815293
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStore.java ---
    @@ -0,0 +1,264 @@
    +/**
    + * 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.daemon.Shutdownable;
    +import backtype.storm.generated.*;
    --- End diff --
    
    intellij feature I hate about, I must change it will do that


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45760090
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStore.java ---
    @@ -0,0 +1,446 @@
    +/**
    + * 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 java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Set;
    +import java.util.regex.Pattern;
    +
    +import javax.security.auth.Subject;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import backtype.storm.daemon.Shutdownable;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +
    +/**
    + * Provides a way to store blobs that can be downloaded.
    + * Blobs must be able to be uploaded and listed from Nimbus,
    + * and downloaded from the Supervisors. It is a key value based
    + * store. Key being a string and value being the blob data.
    + *
    + * ACL checking must take place against the provided subject.
    + * If the blob store does not support Security it must validate
    + * that all ACLs set are always WORLD, everything.
    + *
    + * The users can upload their blobs through the blob store command
    + * line utility. The command line utilty also allows us to update,
    + * delete.
    + *
    + * Modifying the replication factor only works for HdfsBlobStore
    + * as for the LocalFsBlobStore the replication is dependent on
    + * the number of Nimbodes available.
    + */
    +public abstract class BlobStore implements Shutdownable {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStore.class);
    +  private static final Pattern KEY_PATTERN = Pattern.compile("^[\\w \\t\\.:_-]+$");
    +  protected static final String BASE_BLOBS_DIR_NAME = "blobs";
    +
    +  /**
    +   * Allows us to initialize the blob store
    +   * @param conf
    +   * @param baseDir
    +   */
    +  public abstract void prepare(Map conf, String baseDir, NimbusInfo nimbusInfo);
    +
    +  /**
    +   * Creates the blob.
    +   * @param key Key for the blob.
    +   * @param meta Metadata which contains the acls information
    +   * @param who Is the subject creating the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyAlreadyExistsException
    +   */
    +  public abstract AtomicOutputStream createBlob(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException;
    +
    +  /**
    +   * Updates the blob data.
    +   * @param key Key for the blob.
    +   * @param who Is the subject creating the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract AtomicOutputStream updateBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Gets the current version of metadata for a blob
    +   * to be viewed by the user or downloaded by the supervisor.
    +   * @param key Key for the blob.
    +   * @param who Is the subject creating the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract ReadableBlobMeta getBlobMeta(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Sets the metadata with renewed acls for the blob.
    +   * @param key Key for the blob.
    +   * @param meta Metadata which contains the updated
    +   * acls information.
    +   * @param who Is the subject creating the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract void setBlobMeta(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Deletes the blob data and metadata.
    +   * @param key Key for the blob.
    +   * @param who Is the subject creating the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract void deleteBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Gets the current version of metadata for a blob
    +   * to be viewed by the user or downloaded by the supervisor.
    +   * @param key Key for the blob.
    +   * @param who Is the subject creating the blob.
    +   * @return InputStreamWithMeta has the additional
    +   * file length and version information.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract InputStreamWithMeta getBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Returns an iterator with all the list of
    +   * keys currently available on the blob store.
    +   * @param who Is the subject creating the blob.
    +   * @return Iterator<String>
    +   */
    +  public abstract Iterator<String> listKeys(Subject who);
    +
    +  /**
    +   * Gets the replication factor of the blob.
    +   * @param key Key for the blob.
    +   * @param who Is the subject creating the blob.
    +   * @return BlobReplication object containing the
    +   * replication factor for the blob.
    +   * @throws Exception
    +   */
    +  public abstract int getBlobReplication(String key, Subject who) throws Exception;
    --- End diff --
    
    @d2r addressed who param


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45133317
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -372,66 +389,50 @@
              ". State: " state
              ", Heartbeat: " (pr-str heartbeat))
             (shutdown-worker supervisor id)
    -        (if (:code-distributor supervisor)
    -          (.cleanup (:code-distributor supervisor) id))
    -        ))
    -
    -    (doseq [id (vals new-worker-ids)]
    -      (local-mkdirs (worker-pids-root conf id))
    -      (local-mkdirs (worker-heartbeats-root conf id)))
    -    (ls-approved-workers! local-state
    -          (merge
    -           (select-keys (ls-approved-workers local-state)
    -                        (keys keepers))
    -           (zipmap (vals new-worker-ids) (keys new-worker-ids))
    -           ))
    -
    -    ;; check storm topology code dir exists before launching workers
    -    (doseq [[port assignment] reassign-executors]
    -      (let [downloaded-storm-ids (set (read-downloaded-storm-ids conf))
    -            storm-id (:storm-id assignment)
    -            cached-assignment-info @(:assignment-versions supervisor)
    -            assignment-info (if (and (not-nil? cached-assignment-info) (contains? cached-assignment-info storm-id ))
    -                              (get cached-assignment-info storm-id)
    -                              (.assignment-info-with-version storm-cluster-state storm-id nil))
    -	    storm-code-map (read-storm-code-locations assignment-info)
    -            master-code-dir (if (contains? storm-code-map :data) (storm-code-map :data))
    -            stormroot (supervisor-stormdist-root conf storm-id)]
    -        (if-not (or (contains? downloaded-storm-ids storm-id) (.exists (File. stormroot)) (nil? master-code-dir))
    -          (download-storm-code conf storm-id master-code-dir supervisor download-lock))
             ))
    -
    -    (wait-for-workers-launch
    -     conf
    -     (dofor [[port assignment] reassign-executors]
    -            (let [id (new-worker-ids port)
    -                  storm-id (:storm-id assignment)
    -                  ^WorkerResources resources (:resources assignment)
    -                  mem-onheap (.get_mem_on_heap resources)]
    -              (try
    -                (log-message "Launching worker with assignment "
    -                             (pr-str assignment)
    -                             " for this supervisor "
    -                             (:supervisor-id supervisor)
    -                             " on port "
    -                             port
    -                             " with id "
    -                             id
    -                             )
    -                (launch-worker supervisor
    -                               (:storm-id assignment)
    -                               port
    -                               id
    -                               mem-onheap)
    -                (mark! supervisor:num-workers-launched)
    -                (catch java.io.FileNotFoundException e
    -                  (log-message "Unable to launch worker due to "
    -                               (.getMessage e)))
    -                (catch java.io.IOException e
    -                  (log-message "Unable to launch worker due to "
    -                               (.getMessage e))))
    -         id)))
    -    ))
    +    (let [valid-new-worker-ids
    +          (into {}
    +            (remove nil?
    +              (dofor [[port assignment] reassign-executors]
    +                (let [id (new-worker-ids port)
    +                      storm-id (:storm-id assignment)
    +                      ^WorkerResources resources (:resources assignment)
    +                      mem-onheap (.get_mem_on_heap resources)]
    +                  ;; This condition checks for required files exist before launching the worker
    +                  (if (required-topo-files-exist? conf storm-id)
    +                    (do
    +                      (log-message "Launching worker with assignment "
    +                        (pr-str assignment)
    +                        " for this supervisor "
    +                        (:supervisor-id supervisor)
    +                        " on port "
    +                        port
    +                        " with id "
    +                        id)
    +                      (local-mkdirs (worker-pids-root conf id))
    +                      (local-mkdirs (worker-heartbeats-root conf id))
    +                      (launch-worker supervisor
    +                        (:storm-id assignment)
    +                        port
    +                        id
    +                        mem-onheap)
    +                      [port id])
    +                    (do
    +                      (log-message "Missing topology storm code, so can't launch worker with assignment "
    +                        (pr-str assignment)
    +                        " for this supervisor "
    +                        (:supervisor-id supervisor)
    +                        " on port "
    +                        port
    +                        " with id "
    +                        id)
    +                      nil))))))]
    +      (ls-approved-workers! local-state
    +                        (merge
    +                          (select-keys (ls-approved-workers local-state)
    +                            (keys keepers))
    +                          (zipmap (vals valid-new-worker-ids) (keys valid-new-worker-ids))))
    --- End diff --
    
    * Some of the indentation looks odd here.
    * See [above comment](https://github.com/apache/storm/pull/845/files#r45132995) on avoiding map reversal.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45760187
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStore.java ---
    @@ -0,0 +1,446 @@
    +/**
    + * 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 java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Set;
    +import java.util.regex.Pattern;
    +
    +import javax.security.auth.Subject;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import backtype.storm.daemon.Shutdownable;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +
    +/**
    + * Provides a way to store blobs that can be downloaded.
    + * Blobs must be able to be uploaded and listed from Nimbus,
    + * and downloaded from the Supervisors. It is a key value based
    + * store. Key being a string and value being the blob data.
    + *
    + * ACL checking must take place against the provided subject.
    + * If the blob store does not support Security it must validate
    + * that all ACLs set are always WORLD, everything.
    + *
    + * The users can upload their blobs through the blob store command
    + * line. The command line utilty also allows us to update,
    + * delete.
    + *
    + * Modifying the replication factor only works for HdfsBlobStore
    + * as for the LocalFsBlobStore the replication is dependent on
    + * the number of Nimbodes available.
    + */
    +public abstract class BlobStore implements Shutdownable {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStore.class);
    +  private static final Pattern KEY_PATTERN = Pattern.compile("^[\\w \\t\\.:_-]+$");
    +  protected static final String BASE_BLOBS_DIR_NAME = "blobs";
    +
    +  /**
    +   * Allows us to initialize the blob store
    +   * @param conf The storm configuration
    +   * @param baseDir The directory path to store the blobs
    +   * @param nimbusInfo Contains the nimbus host, port and leadership information.
    +   */
    +  public abstract void prepare(Map conf, String baseDir, NimbusInfo nimbusInfo);
    +
    +  /**
    +   * Creates the blob.
    +   * @param key Key for the blob.
    +   * @param meta Metadata which contains the acls information
    +   * @param who Is the subject creating the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyAlreadyExistsException
    +   */
    +  public abstract AtomicOutputStream createBlob(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException;
    +
    +  /**
    +   * Updates the blob data.
    +   * @param key Key for the blob.
    +   * @param who Is the subject having the write privilege for the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract AtomicOutputStream updateBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Gets the current version of metadata for a blob
    +   * to be viewed by the user or downloaded by the supervisor.
    +   * @param key Key for the blob.
    +   * @param who Is the subject having the read privilege for the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract ReadableBlobMeta getBlobMeta(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Sets the metadata with renewed acls for the blob.
    +   * @param key Key for the blob.
    +   * @param meta Metadata which contains the updated
    +   * acls information.
    +   * @param who Is the subject having the write privilege for the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract void setBlobMeta(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Deletes the blob data and metadata.
    +   * @param key Key for the blob.
    +   * @param who Is the subject having write privilege for the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract void deleteBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Gets the InputStream to read the blob details
    +   * @param key Key for the blob.
    +   * @param who Is the subject having the read privilege for the blob.
    +   * @return InputStreamWithMeta has the additional
    +   * file length and version information.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract InputStreamWithMeta getBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Returns an iterator with all the list of
    +   * keys currently available on the blob store.
    +   * @param who Is the subject creating the blob.
    +   * @return Iterator<String>
    +   */
    +  public abstract Iterator<String> listKeys(Subject who);
    +
    +  /**
    +   * Gets the replication factor of the blob.
    +   * @param key Key for the blob.
    +   * @param who Is the subject having the read privilege for the blob.
    +   * @return BlobReplication object containing the
    +   * replication factor for the blob.
    +   * @throws Exception
    +   */
    +  public abstract int getBlobReplication(String key, Subject who) throws Exception;
    +
    +  /**
    +   * Modifies the replication factor of the blob.
    +   * @param key Key for the blob.
    +   * @param replication The replication factor the
    +   * blob has to be set to.
    +   * @param who Is the subject having the update privilege for the blob
    +   * @return BlobReplication object containing the
    +   * updated replication factor for the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   * @throws IOException
    +   */
    +  public abstract int updateBlobReplication(String key, int replication, Subject who) throws AuthorizationException, KeyNotFoundException, IOException;
    +
    +  /**
    +   * Filters keys based on the KeyFilter
    +   * passed as the argument.
    +   * @param filter Filter passed as the
    +   * @param who Might not want to have the subject as it is not doing anything
    --- End diff --
    
    @d2r addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45106345
  
    --- 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 {
    --- End diff --
    
    "BlobSynchronizer"?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43934358
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -346,53 +400,104 @@
           [(.getNodeId slot) (.getPort slot)]
           )))
     
    +(defn- get-metadata-version [blob-store key subject]
    +  (let [blob-meta (.getBlobMeta blob-store key subject)]
    +    (.get_version blob-meta)))
    +
    +(defn get-key-set-from-blob-store [blob-store]
    +  (let [key-iter (.listKeys blob-store get-nimbus-subject)]
    +    (set (iterator-seq key-iter))))
    +
     (defn- setup-storm-code [nimbus conf storm-id tmp-jar-location storm-conf topology]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -   (log-message "nimbus file location:" stormroot)
    -   (FileUtils/forceMkdir (File. stormroot))
    -   (FileUtils/cleanDirectory (File. stormroot))
    -   (setup-jar conf tmp-jar-location stormroot)
    -   (FileUtils/writeByteArrayToFile (File. (master-stormcode-path stormroot)) (Utils/serialize topology))
    -   (FileUtils/writeByteArrayToFile (File. (master-stormconf-path stormroot)) (Utils/toCompressedJsonConf storm-conf))
    -   (if (:code-distributor nimbus) (.upload (:code-distributor nimbus) stormroot storm-id))
    -   ))
    +  (let [subject (get-subject)
    +        storm-cluster-state (:storm-cluster-state nimbus)
    +        blob-store (:blob-store nimbus)
    +        jar-key (master-stormjar-key storm-id)
    +        code-key (master-stormcode-key storm-id)
    +        conf-key (master-stormconf-key storm-id)
    +        nimbus-host-port-info (:nimbus-host-port-info nimbus)]
    +    (if tmp-jar-location ;;in local mode there is no jar
    +      (do
    +        (.createBlob blob-store jar-key (FileInputStream. tmp-jar-location) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +        (if (instance? LocalFsBlobStore blob-store)
    +          (.setup-blobstore! storm-cluster-state jar-key nimbus-host-port-info (get-metadata-version blob-store jar-key subject)))))
    +    (.createBlob blob-store conf-key (Utils/toCompressedJsonConf storm-conf) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state conf-key nimbus-host-port-info (get-metadata-version blob-store conf-key subject)))
    +    (.createBlob blob-store code-key (Utils/serialize topology) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state code-key nimbus-host-port-info (get-metadata-version blob-store code-key subject)))))
    +
    +(defn- read-storm-topology [storm-id blob-store]
    +  (Utils/deserialize
    +    (.readBlob blob-store (master-stormcode-key storm-id) (get-subject)) StormTopology))
    +
    +(defn- get-blob-replication-count [blob-key nimbus]
    +  (if (:blob-store nimbus)
    +          (-> (:blob-store nimbus)
    +            (.getBlobReplication  blob-key get-nimbus-subject)
    +            (.get_replication))))
     
     (defn- wait-for-desired-code-replication [nimbus conf storm-id]
       (let [min-replication-count (conf TOPOLOGY-MIN-REPLICATION-COUNT)
             max-replication-wait-time (conf TOPOLOGY-MAX-REPLICATION-WAIT-TIME-SEC)
    -        total-wait-time (atom 0)
    -        current-replication-count (atom (if (:code-distributor nimbus) (.getReplicationCount (:code-distributor nimbus) storm-id) 0))]
    -  (if (:code-distributor nimbus)
    -    (while (and (> min-replication-count @current-replication-count)
    -             (or (= -1 max-replication-wait-time)
    -               (< @total-wait-time max-replication-wait-time)))
    +        current-replication-count-jar (if (not (local-mode? conf)) (atom
    +                                                                     (get-blob-replication-count (master-stormjar-key storm-id) nimbus))
    +                                                                     (atom min-replication-count))
    +        current-replication-count-code (atom (get-blob-replication-count (master-stormcode-key storm-id) nimbus))
    +        current-replication-count-conf (atom (get-blob-replication-count (master-stormconf-key storm-id) nimbus))
    +        total-wait-time (atom 0)]
    +    (log-message "wait for desired replication" "count"
    +      min-replication-count "wait-time" max-replication-wait-time
    +      "code" @current-replication-count-code
    +      "conf" @current-replication-count-conf
    +      "jar" @current-replication-count-jar
    +      "replication count" (get-blob-replication-count (master-stormconf-key storm-id) nimbus))
    +    (if (:blob-store nimbus)
    +      (while (and (> min-replication-count @current-replication-count-jar)
    +               (> min-replication-count @current-replication-count-code)
    +               (> min-replication-count @current-replication-count-conf)
    +               (or (= -1 max-replication-wait-time)
    +                 (< @total-wait-time max-replication-wait-time)))
             (sleep-secs 1)
             (log-debug "waiting for desired replication to be achieved.
               min-replication-count = " min-replication-count  " max-replication-wait-time = " max-replication-wait-time
    -          "current-replication-count = " @current-replication-count " total-wait-time " @total-wait-time)
    +          (if (not (local-mode? conf))"current-replication-count for jar key = " @current-replication-count-jar)
    +          "current-replication-count for code key = " @current-replication-count-code
    +          "current-replication-count for conf key = " @current-replication-count-conf
    +          " total-wait-time " @total-wait-time)
             (swap! total-wait-time inc)
    -        (reset! current-replication-count  (.getReplicationCount (:code-distributor nimbus) storm-id))))
    -  (if (< min-replication-count @current-replication-count)
    -    (log-message "desired replication count "  min-replication-count " achieved,
    -      current-replication-count" @current-replication-count)
    -    (log-message "desired replication count of "  min-replication-count " not achieved but we have hit the max wait time "
    -      max-replication-wait-time " so moving on with replication count = " @current-replication-count)
    -    )))
    -
    -(defn- read-storm-topology [conf storm-id]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -    (Utils/deserialize
    -      (FileUtils/readFileToByteArray
    -        (File. (master-stormcode-path stormroot))
    -        ) StormTopology)))
    +        (if (not (local-mode? conf))(reset! current-replication-count-conf  (get-blob-replication-count (master-stormconf-key storm-id))))
    +          (reset! current-replication-count-code  (get-blob-replication-count (master-stormcode-key storm-id)))
    +          (reset! current-replication-count-jar  (get-blob-replication-count (master-stormjar-key storm-id)))))
    --- End diff --
    
    `get-blob-replication-count` is defined above with arity 2, but we are calling it three times here with one argument.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on the pull request:

    https://github.com/apache/storm/pull/845#issuecomment-157180992
  
    //BLOB APIs
        // These blob APIs guarantee very little.
        // Writes and reads can fail at any point in time and should be retried.
        // deletes are a best effort and if someone is adding or updating the same key
        //  at the same time the key may exist afterwards.
        // About the only thing that is guaranteed is a blob will be self consistent.
        // When downloading a blob all of the bits will be from the same version of the
        // blob. It may even mean that reading throws an exception in the middle.
        // Many of the APIs have sessions assoicated with them.  If you take too long to
        // complete a session it may timeout and you will need to start over.
    
    @derek -> Removed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45759993
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStore.java ---
    @@ -0,0 +1,446 @@
    +/**
    + * 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 java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Set;
    +import java.util.regex.Pattern;
    +
    +import javax.security.auth.Subject;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import backtype.storm.daemon.Shutdownable;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +
    +/**
    + * Provides a way to store blobs that can be downloaded.
    + * Blobs must be able to be uploaded and listed from Nimbus,
    + * and downloaded from the Supervisors. It is a key value based
    + * store. Key being a string and value being the blob data.
    + *
    + * ACL checking must take place against the provided subject.
    + * If the blob store does not support Security it must validate
    + * that all ACLs set are always WORLD, everything.
    + *
    + * The users can upload their blobs through the blob store command
    + * line utility. The command line utilty also allows us to update,
    + * delete.
    + *
    + * Modifying the replication factor only works for HdfsBlobStore
    + * as for the LocalFsBlobStore the replication is dependent on
    + * the number of Nimbodes available.
    + */
    +public abstract class BlobStore implements Shutdownable {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStore.class);
    +  private static final Pattern KEY_PATTERN = Pattern.compile("^[\\w \\t\\.:_-]+$");
    +  protected static final String BASE_BLOBS_DIR_NAME = "blobs";
    +
    +  /**
    +   * Allows us to initialize the blob store
    +   * @param conf
    +   * @param baseDir
    +   */
    --- End diff --
    
    @d2r addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46100239
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/NimbusBlobStore.java ---
    @@ -0,0 +1,419 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.BeginDownloadResult;
    +import backtype.storm.generated.ListBlobsResult;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.utils.NimbusClient;
    +import backtype.storm.utils.Utils;
    +import org.apache.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.nio.ByteBuffer;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +
    +public class NimbusBlobStore extends ClientBlobStore {
    +  private static final Logger LOG = LoggerFactory.getLogger(NimbusBlobStore.class);
    +
    +  public class NimbusKeyIterator implements Iterator<String> {
    +    private ListBlobsResult listBlobs = null;
    +    private int offset = 0;
    +    private boolean eof = false;
    +    
    +    public NimbusKeyIterator(ListBlobsResult listBlobs) {
    +      this.listBlobs = listBlobs;
    +      this.eof = (listBlobs.get_keys_size() == 0);
    +    }
    +    
    +    private boolean isCacheEmpty() {
    +      return listBlobs.get_keys_size() <= offset;
    +    }
    +
    +    private void readMore() {
    +      if (!eof) {
    +        try {
    +          offset = 0;
    +          synchronized(client) {
    +            listBlobs = client.getClient().listBlobs(listBlobs.get_session());
    +          }
    +          if (listBlobs.get_keys_size() == 0) {
    +            eof = true;
    +          }
    +        } catch (TException e) {
    +          throw new RuntimeException(e);
    +        }
    +      }
    +    }
    +    
    +    @Override
    +    public synchronized boolean hasNext() {
    +      if (isCacheEmpty()) {
    +        readMore();
    +      }
    +      return !eof;
    +    }
    +
    +    @Override
    +    public synchronized String next() {
    +      if (!hasNext()) {
    +        throw new NoSuchElementException();
    +      }
    +      String ret = listBlobs.get_keys().get(offset);
    +      offset++;
    +      return ret;
    +    }
    +    
    +    @Override
    +    public void remove() {
    +      throw new UnsupportedOperationException("Delete Not Supported");
    +    }
    +  }
    +
    +  public class NimbusDownloadInputStream extends InputStreamWithMeta {
    +    private BeginDownloadResult beginBlobDownload;
    +    private byte[] buffer = null;
    +    private int offset = 0;
    +    private int end = 0;
    +    private boolean eof = false;
    +
    +    public NimbusDownloadInputStream(BeginDownloadResult beginBlobDownload) {
    +      this.beginBlobDownload = beginBlobDownload;
    +    }
    +
    +    @Override
    +    public long getVersion() throws IOException {
    +      return beginBlobDownload.get_version();
    +    }
    +
    +    @Override
    +    public synchronized int read() throws IOException {
    +      if (isEmpty()) {
    +        readMore();
    +        if (eof) {
    +          return -1;
    +        }
    +      }
    +      int length = Math.min(1, available());
    +      if (length == 0) {
    +        return -1;
    +      }
    +      int ret = buffer[offset];
    +      offset += length;
    +      return ret;
    +    }
    +    
    +    @Override 
    +    public synchronized int read(byte[] b, int off, int len) throws IOException {
    +      if (isEmpty()) {
    +        readMore();
    --- End diff --
    
    Very first call to readMore when buffer is null makes sense but subsequent calls to read when buffer is not null require an array copy, it might be tedious to refactor some of the code used at other places


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r44997103
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -526,6 +605,41 @@
       {Config/SUPERVISOR_MEMORY_CAPACITY_MB (double (conf SUPERVISOR-MEMORY-CAPACITY-MB))
        Config/SUPERVISOR_CPU_CAPACITY (double (conf SUPERVISOR-CPU-CAPACITY))})
     
    +(defn update-blobs-for-topology!
    +  "Update each blob listed in the topology configuration if the latest version of the blob
    +   has not been downloaded."
    +  [conf storm-id localizer]
    +  (let [storm-conf (read-supervisor-storm-conf conf storm-id)
    +        blobstore-map (storm-conf TOPOLOGY-BLOBSTORE-MAP)
    +        user (storm-conf TOPOLOGY-SUBMITTER-USER)
    +        topo-name (storm-conf TOPOLOGY-NAME)
    +        user-dir (.getLocalUserFileCacheDir localizer user)
    +        localresources (blobstore-map-to-localresources blobstore-map)]
    +    (try
    +      (.updateBlobs localizer localresources user)
    +      (catch AuthorizationException authExp
    +        (log-error authExp))
    +      (catch KeyNotFoundException knf
    +        (log-error knf)))))
    +
    +(defn update-blobs-for-all-topologies-fn
    +  "Returns a function that downloads all blobs listed in the topology configuration for all topologies assigned
    +  to this supervisor, and creates version files with a suffix. The returned function is intended to be run periodically
    +  by a timer, created elsewhere."
    +  [supervisor]
    +  (fn this []
    --- End diff --
    
    Don't name the function 'this'. Either give it a meaningful name or get rid of the name.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45429107
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/KeyVersion.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * 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.zookeeper.CreateMode;
    +import org.apache.zookeeper.ZooDefs;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.TreeSet;
    +import java.util.Map;
    +import java.util.List;
    +
    +/**
    + * Class hands over the version of the key to be stored within the zookeeper
    + */
    +public class KeyVersion {
    +  private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
    +  private final String BLOBSTORE_SUBTREE="/blobstore";
    +  private final String BLOBSTORE_KEY_COUNTER_SUBTREE="/blobstorekeycounter";
    +  private String key;
    +  private NimbusInfo nimbusInfo;
    +
    +  public KeyVersion(String key, NimbusInfo nimbusInfo) {
    +    this.key = key;
    +    this.nimbusInfo = nimbusInfo;
    +  }
    +
    +  public int getKeyVersion(Map conf) {
    +    TreeSet<Integer> versions = new TreeSet<Integer>();
    +    CuratorFramework zkClient = Utils.createZKClient(conf);
    +    try {
    +      // Key has not been created yet and it is the first time it is being created
    +      if(zkClient.checkExists().forPath(BLOBSTORE_SUBTREE + "/" + key) == null) {
    +        zkClient.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT)
    +                .withACL(ZooDefs.Ids.OPEN_ACL_UNSAFE).forPath(BLOBSTORE_KEY_COUNTER_SUBTREE + "/" + key + "/" + 1);
    +        return 1;
    +      }
    +
    +      // When all nimbodes go down and one or few of them come up
    +      // Unfortunately there might not be an exact way to know which one contains the most updated blob
    +      // if all go down which is unlikely. Hence there might be a need to update the blob if all go down
    +      List<String> stateInfoList = zkClient.getChildren().forPath(BLOBSTORE_SUBTREE + "/" + key);
    +      LOG.debug("stateInfoList {} stateInfoList {}", stateInfoList.size(), stateInfoList);
    +      if(stateInfoList.isEmpty()) {
    +        return getKeyVersionCounterValue(zkClient, key);
    +      }
    +
    +      LOG.debug("stateInfoSize {}", stateInfoList.size());
    +      // In all other cases check for the latest version on the nimbus and assign the version
    +      // check if all are have same version, if not assign the highest version
    +      for (String stateInfo:stateInfoList) {
    +        versions.add(Integer.parseInt(Utils.normalizeVersionInfo(stateInfo)[1]));
    +      }
    +
    +      int currentCounter = getKeyVersionCounterValue(zkClient, key);
    +      // This condition returns version when a nimbus crashes and comes up
    +      if (!checkIfStateContainsCurrentNimbusHost(stateInfoList, nimbusInfo) && !nimbusInfo.isLeader()) {
    +        if (versions.last() < currentCounter) {
    +          return currentCounter;
    +        } else {
    +          return currentCounter - 1;
    +        }
    +      }
    +      // Condition checks for an update scenario
    +      if (stateInfoList.size() >= 1 && versions.size() == 1) {
    --- End diff --
    
    If `versions.size()` is `1`, it is redundant to check for `stateInfoList.size()` since it must be 1 or greater. We don't add to `versions` unless there is an element in `stateInfoList`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on the pull request:

    https://github.com/apache/storm/pull/845#issuecomment-153483796
  
    storm.thrift:
    ```
    struct BlobReplication {
    1: required i32 replication;
    }
    ```
    
    It would be good to get rid of this and just use `i32` if we can.
    
    If we cannot, let's add a comment explaining why it is needed.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45561642
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/FileBlobStoreImpl.java ---
    @@ -0,0 +1,248 @@
    +/**
    + * 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.Config;
    +import backtype.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.FileNotFoundException;
    +import java.io.IOException;
    +import java.nio.file.Files;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Timer;
    +import java.util.TimerTask;
    +
    +/**
    + * Very basic blob store impl with no ACL handling.
    + */
    +public class FileBlobStoreImpl {
    +  private static final long FULL_CLEANUP_FREQ = 60 * 60 * 1000l;
    +  private static final int BUCKETS = 1024;
    +  private static final Logger LOG = LoggerFactory.getLogger(FileBlobStoreImpl.class);
    +  private static final Timer timer = new Timer("FileBlobStore cleanup thread", true);
    +
    +  public class KeyInHashDirIterator implements Iterator<String> {
    +    private int currentBucket = 0;
    +    private Iterator<String> it = null;
    +    private String next = null;
    +
    +    public KeyInHashDirIterator() throws IOException {
    +      primeNext();
    +    }
    +
    +    private void primeNext() throws IOException {
    +      while (it == null && currentBucket < BUCKETS) {
    +        String name = String.valueOf(currentBucket);
    +        File dir = new File(_fullPath, name);
    +        try {
    +          it = listKeys(dir);
    +        } catch (FileNotFoundException e) {
    +          it = null;
    +        }
    +        if (it == null || !it.hasNext()) {
    +          it = null;
    +          currentBucket++;
    +        } else {
    +          next = it.next();
    +        }
    +      }
    +    }
    +
    +    @Override
    +    public boolean hasNext() {
    +      return next != null;
    +    }
    +
    +    @Override
    +    public String next() {
    +      if (!hasNext()) {
    +        throw new NoSuchElementException();
    +      }
    +      String current = next;
    +      next = null;
    +      if (it != null) {
    +        if (!it.hasNext()) {
    +          it = null;
    +          currentBucket++;
    +          try {
    +            primeNext();
    +          } catch (IOException e) {
    +            throw new RuntimeException(e);
    +          }
    +        } else {
    +          next = it.next();
    +        }
    +      }
    +      return current;
    +    }
    +
    +    @Override
    +    public void remove() {
    +      throw new UnsupportedOperationException("Delete Not Supported");
    +    }
    +  }
    +
    +  private File _fullPath;
    +  private TimerTask cleanup = null;
    +
    +  public FileBlobStoreImpl(File path, Map<String, Object> conf) throws IOException {
    +    LOG.info("Creating new blob store based in {}", path);
    +    _fullPath = path;
    +    _fullPath.mkdirs();
    +    Object shouldCleanup = conf.get(Config.BLOBSTORE_CLEANUP_ENABLE);
    +    if (Utils.getBoolean(shouldCleanup, false)) {
    +      LOG.debug("Starting File blobstore cleaner");
    +      cleanup = new TimerTask() {
    +        @Override
    +        public void run() {
    +          try {
    +            fullCleanup(FULL_CLEANUP_FREQ);
    +          } catch (IOException e) {
    +            LOG.error("Error trying to cleanup", e);
    +          }
    +        }
    +      };
    +      timer.scheduleAtFixedRate(cleanup, 0, FULL_CLEANUP_FREQ);
    +    }
    +  }
    +
    +  /**
    +   * @return all keys that are available for reading.
    +   * @throws IOException on any error.
    +   */ 
    +  public Iterator<String> listKeys() throws IOException {
    +    return new KeyInHashDirIterator();
    +  }
    +
    +  /**
    +   * Get an input stream for reading a part.
    +   * @param key the key of the part to read.
    +   * @return the where to read the data from.
    +   * @throws IOException on any error
    +   */
    +  public LocalFsBlobStoreFile read(String key) throws IOException {
    +    return new LocalFsBlobStoreFile(getKeyDir(key), BlobStoreFile.BLOBSTORE_DATA_FILE);
    +  }
    +
    +  /**
    +   * Get an object tied to writing the data.
    +   * @param key the key of the part to write to.
    +   * @return an object that can be used to both write to, but also commit/cancel the operation.
    +   * @throws IOException on any error
    +   */ 
    +  public LocalFsBlobStoreFile write(String key, boolean create) throws IOException {
    +    return new LocalFsBlobStoreFile(getKeyDir(key), true, create);
    +  }
    +
    +  /**
    +   * Check if the key exists in the blob store.
    +   * @param key the key to check for
    +   * @return true if it exists else false.
    +   */
    +  public boolean exists(String key) {
    +    return getKeyDir(key).exists();
    +  }
    +  
    +  /**
    +   * Delete a key from the blob store
    +   * @param key the key to delete
    +   * @throws IOException on any error
    +   */
    +  public void deleteKey(String key) throws IOException {
    +    File keyDir = getKeyDir(key);
    +    LocalFsBlobStoreFile pf = new LocalFsBlobStoreFile(keyDir, BlobStoreFile.BLOBSTORE_DATA_FILE);
    +    pf.delete();
    +    delete(keyDir);
    +  }
    +
    +  private File getKeyDir(String key) {
    +    String hash = String.valueOf(Math.abs((long)key.hashCode()) % BUCKETS);
    +    File ret = new File(new File(_fullPath, hash), key);
    +    LOG.debug("{} Looking for {} in {}", new Object[]{_fullPath, key, hash});
    +    return ret;
    +  }
    +
    +  public void fullCleanup(long age) throws IOException {
    +    long cleanUpIfBefore = System.currentTimeMillis() - age;
    +    Iterator<String> keys = new KeyInHashDirIterator();
    +    while (keys.hasNext()) {
    +      String key = keys.next();
    +      File keyDir = getKeyDir(key);
    +      Iterator<LocalFsBlobStoreFile> i = listBlobStoreFiles(keyDir);
    +      if (!i.hasNext()) {
    +        //The dir is empty, so try to delete it, may fail, but that is OK
    +        try {
    +          keyDir.delete();
    +        } catch (Exception e) {
    +          LOG.warn("Could not delete "+keyDir+" will try again later");
    +        }
    +      }
    +      while (i.hasNext()) {
    +        LocalFsBlobStoreFile f = i.next();
    +        if (f.isTmp()) {
    +          if (f.getModTime() <= cleanUpIfBefore) {
    +            f.delete();
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  protected Iterator<LocalFsBlobStoreFile> listBlobStoreFiles(File path) throws IOException {
    +    ArrayList<LocalFsBlobStoreFile> ret = new ArrayList<LocalFsBlobStoreFile>();
    +    File[] files = path.listFiles();
    +    if (files != null) {
    --- End diff --
    
    I think it does
    @d2r 
    public File[] listFiles() {
            String[] ss = list();
            if (ss == null) return null;
    The list() here might return null
           public String[] list() {
            SecurityManager security = System.getSecurityManager();
            if (security != null) {
                security.checkRead(path);
            }
            if (isInvalid()) {
                return null;
            }


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by kishorvpatil <gi...@git.apache.org>.
Github user kishorvpatil commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43811863
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -950,20 +1047,33 @@
                 TOPOLOGY-EVENTLOGGER-EXECUTORS (total-conf TOPOLOGY-EVENTLOGGER-EXECUTORS)
                 TOPOLOGY-MAX-TASK-PARALLELISM (total-conf TOPOLOGY-MAX-TASK-PARALLELISM)})))
     
    +(defn blob-rm-key [blob-store key storm-cluster-state]
    +  (try
    +    (.deleteBlob blob-store key (get-nimbus-subject))
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.remove-blobstore-key! storm-cluster-state key))
    +    (catch Exception e)))
    --- End diff --
    
    should not ignore exception. At least log it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45433397
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStore.java ---
    @@ -0,0 +1,306 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.utils.Utils;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.thrift.TBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.ByteArrayOutputStream;
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.FileNotFoundException;
    +import java.io.InputStream;
    +
    +import java.util.*;
    +
    +import static backtype.storm.blobstore.BlobStoreAclHandler.ADMIN;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.READ;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.WRITE;
    +
    +/**
    + * Provides a local file system backed blob store implementation for Nimbus.
    + */
    +public class LocalFsBlobStore extends BlobStore {
    +  public static final Logger LOG = LoggerFactory.getLogger(LocalFsBlobStore.class);
    +  private static final String DATA_PREFIX = "data_";
    +  private static final String META_PREFIX = "meta_";
    +  protected BlobStoreAclHandler _aclHandler;
    +  private final String BLOBSTORE_SUBTREE = "/blobstore/";
    +  private NimbusInfo nimbusInfo;
    +  private FileBlobStoreImpl fbs;
    +  private Map conf;
    +
    +  @Override
    +  public void prepare(Map conf, String overrideBase, NimbusInfo nimbusInfo) {
    +    this.conf = conf;
    +    this.nimbusInfo = nimbusInfo;
    +     if (overrideBase == null) {
    +      overrideBase = (String)conf.get(Config.BLOBSTORE_DIR);
    +      if (overrideBase == null) {
    +        overrideBase = (String) conf.get(Config.STORM_LOCAL_DIR);
    +      }
    +    }
    +    File baseDir = new File(overrideBase, BASE_BLOBS_DIR_NAME);
    +    try {
    +      fbs = new FileBlobStoreImpl(baseDir, conf);
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +    _aclHandler = new BlobStoreAclHandler(conf);
    +  }
    +
    +  @Override
    +  public AtomicOutputStream createBlob(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException {
    +    LOG.debug("Creating Blob for key {}", key);
    +    validateKey(key);
    +    _aclHandler.normalizeSettableBlobMeta(key, meta, who, READ | WRITE | ADMIN);
    +    BlobStoreAclHandler.validateSettableACLs(key, meta.get_acl());
    +    _aclHandler.hasPermissions(meta.get_acl(), READ | WRITE | ADMIN, who, key);
    +    if (fbs.exists(DATA_PREFIX+key)) {
    +      throw new KeyAlreadyExistsException(key);
    +    }
    +    BlobStoreFileOutputStream mOut = null;
    +    try {
    +      mOut = new BlobStoreFileOutputStream(fbs.write(META_PREFIX+key, true));
    +      mOut.write(Utils.thriftSerialize((TBase) meta));
    +      mOut.close();
    +      mOut = null;
    +      return new BlobStoreFileOutputStream(fbs.write(DATA_PREFIX+key, true));
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    } finally {
    +      if (mOut != null) {
    +        try {
    +          mOut.cancel();
    +        } catch (IOException e) {
    +          //Ignored
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public AtomicOutputStream updateBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    validateKey(key);
    +    checkForBlobOrDownload(key);
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(meta.get_acl(), WRITE, who, key);
    +    try {
    +      return new BlobStoreFileOutputStream(fbs.write(DATA_PREFIX+key, false));
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  private SettableBlobMeta getStoredBlobMeta(String key) throws KeyNotFoundException {
    +    InputStream in = null;
    +    try {
    +      LocalFsBlobStoreFile pf = fbs.read(META_PREFIX+key);
    +      try {
    +        in = pf.getInputStream();
    +      } catch (FileNotFoundException fnf) {
    +        throw new KeyNotFoundException(key);
    +      }
    +      ByteArrayOutputStream out = new ByteArrayOutputStream();
    +      byte [] buffer = new byte[2048];
    +      int len;
    +      while ((len = in.read(buffer)) > 0) {
    +        out.write(buffer, 0, len);
    +      }
    +      in.close();
    +      in = null;
    +      SettableBlobMeta sbm = Utils.thriftDeserialize(SettableBlobMeta.class, out.toByteArray());
    +      return sbm;
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    } finally {
    +      if (in != null) {
    +        try {
    +          in.close();
    +        } catch (IOException e) {
    +          //Ignored
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public ReadableBlobMeta getBlobMeta(String key, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    validateKey(key);
    +    if(!checkForBlobOrDownload(key)) {
    +      checkForBlobUpdate(key);
    +    }
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.validateUserCanReadMeta(meta.get_acl(), who, key);
    +    ReadableBlobMeta rbm = new ReadableBlobMeta();
    +    rbm.set_settable(meta);
    +    try {
    +      LocalFsBlobStoreFile pf = fbs.read(DATA_PREFIX+key);
    +      rbm.set_version(pf.getModTime());
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +    return rbm;
    +  }
    +
    +  @Override
    +  public void setBlobMeta(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    validateKey(key);
    +    checkForBlobOrDownload(key);
    +    _aclHandler.normalizeSettableBlobMeta(key, meta, who, ADMIN);
    +    BlobStoreAclHandler.validateSettableACLs(key, meta.get_acl());
    +    SettableBlobMeta orig = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(orig.get_acl(), ADMIN, who, key);
    +    BlobStoreFileOutputStream mOut = null;
    +    try {
    +      mOut = new BlobStoreFileOutputStream(fbs.write(META_PREFIX+key, false));
    +      mOut.write(Utils.thriftSerialize((TBase) meta));
    +      mOut.close();
    +      mOut = null;
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    } finally {
    +      if (mOut != null) {
    +        try {
    +          mOut.cancel();
    +        } catch (IOException e) {
    +          //Ignored
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void deleteBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    validateKey(key);
    +    checkForBlobOrDownload(key);
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(meta.get_acl(), WRITE, who, key);
    +    try {
    +      fbs.deleteKey(DATA_PREFIX+key);
    +      fbs.deleteKey(META_PREFIX+key);
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override
    +  public InputStreamWithMeta getBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    validateKey(key);
    +    if(!checkForBlobOrDownload(key)) {
    +      checkForBlobUpdate(key);
    +    }
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(meta.get_acl(), READ, who, key);
    +    try {
    +      return new BlobStoreFileInputStream(fbs.read(DATA_PREFIX+key));
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override
    +  public Iterator<String> listKeys(Subject who) {
    +    try {
    +      return new KeyTranslationIterator(fbs.listKeys(), DATA_PREFIX);
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override
    +  public void shutdown() {
    +  }
    +
    +  @Override
    +  public int getBlobReplication(String key, Subject who) throws Exception {
    +    validateKey(key);
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(meta.get_acl(), READ, who, key);
    +    CuratorFramework zkClient = Utils.createZKClient(conf);
    +    if (zkClient.checkExists().forPath(BLOBSTORE_SUBTREE + key) == null) {
    +       zkClient.close();
    +       return 0;
    +    }
    +    int replicationCount = zkClient.getChildren().forPath(BLOBSTORE_SUBTREE + key).size();
    +    zkClient.close();
    +    return replicationCount;
    +  }
    +
    +  @Override
    +  public int updateBlobReplication(String key, int replication, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    int replicationCount = 0;
    +    validateKey(key);
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(meta.get_acl(), WRITE, who, key);
    +    try {
    +      LOG.warn ("For local file system blob store the update blobs function does not work." +
    +                "Please use HDFS blob store to make this feature available. The replication your " +
    +                "are noticing is the present replication of the blob based on its availability on various nimbuses");
    --- End diff --
    
    Should we instead throw UnsupportedOperationException? This message will go to the nimbus log, and if the user were to attempt to call this, it would appear to have no effect without reason.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on the pull request:

    https://github.com/apache/storm/pull/845#issuecomment-153492370
  
    storm.thrift:
    
    ```
      //BLOB APIs
        // These blob APIs guarantee very little.
        // Writes and reads can fail at any point in time and should be retried.
        // deletes are a best effort and if someone is adding or updating the same key
        //  at the same time the key may exist afterwards.
        // About the only thing that is guaranteed is a blob will be self consistent.
        // When downloading a blob all of the bits will be from the same version of the
        // blob. It may even mean that reading throws an exception in the middle.
        // Many of the APIs have sessions assoicated with them.  If you take too long to
        // complete a session it may timeout and you will need to start over.
        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);
        //can stop downloading chunks when receive 0-length byte array back
        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); //empty string "" means start at the beginning
        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);
        void createStateInZookeeper(1: string key); // creates state in zookeeper when blob is uploaded through command line
    ```
    
    * all but the first line is indented 2 extra spaces
    * Should we move the comments to a readme?
    * "These blob APIs guarantee" -> This blob API guarantees
    * "assoicated" -> associated
    * "deletes" -> Deletes



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43935130
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -346,53 +400,104 @@
           [(.getNodeId slot) (.getPort slot)]
           )))
     
    +(defn- get-metadata-version [blob-store key subject]
    +  (let [blob-meta (.getBlobMeta blob-store key subject)]
    +    (.get_version blob-meta)))
    +
    +(defn get-key-set-from-blob-store [blob-store]
    +  (let [key-iter (.listKeys blob-store get-nimbus-subject)]
    +    (set (iterator-seq key-iter))))
    +
     (defn- setup-storm-code [nimbus conf storm-id tmp-jar-location storm-conf topology]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -   (log-message "nimbus file location:" stormroot)
    -   (FileUtils/forceMkdir (File. stormroot))
    -   (FileUtils/cleanDirectory (File. stormroot))
    -   (setup-jar conf tmp-jar-location stormroot)
    -   (FileUtils/writeByteArrayToFile (File. (master-stormcode-path stormroot)) (Utils/serialize topology))
    -   (FileUtils/writeByteArrayToFile (File. (master-stormconf-path stormroot)) (Utils/toCompressedJsonConf storm-conf))
    -   (if (:code-distributor nimbus) (.upload (:code-distributor nimbus) stormroot storm-id))
    -   ))
    +  (let [subject (get-subject)
    +        storm-cluster-state (:storm-cluster-state nimbus)
    +        blob-store (:blob-store nimbus)
    +        jar-key (master-stormjar-key storm-id)
    +        code-key (master-stormcode-key storm-id)
    +        conf-key (master-stormconf-key storm-id)
    +        nimbus-host-port-info (:nimbus-host-port-info nimbus)]
    +    (if tmp-jar-location ;;in local mode there is no jar
    +      (do
    +        (.createBlob blob-store jar-key (FileInputStream. tmp-jar-location) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +        (if (instance? LocalFsBlobStore blob-store)
    +          (.setup-blobstore! storm-cluster-state jar-key nimbus-host-port-info (get-metadata-version blob-store jar-key subject)))))
    +    (.createBlob blob-store conf-key (Utils/toCompressedJsonConf storm-conf) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state conf-key nimbus-host-port-info (get-metadata-version blob-store conf-key subject)))
    +    (.createBlob blob-store code-key (Utils/serialize topology) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state code-key nimbus-host-port-info (get-metadata-version blob-store code-key subject)))))
    +
    +(defn- read-storm-topology [storm-id blob-store]
    +  (Utils/deserialize
    +    (.readBlob blob-store (master-stormcode-key storm-id) (get-subject)) StormTopology))
    +
    +(defn- get-blob-replication-count [blob-key nimbus]
    +  (if (:blob-store nimbus)
    +          (-> (:blob-store nimbus)
    +            (.getBlobReplication  blob-key get-nimbus-subject)
    +            (.get_replication))))
     
     (defn- wait-for-desired-code-replication [nimbus conf storm-id]
       (let [min-replication-count (conf TOPOLOGY-MIN-REPLICATION-COUNT)
             max-replication-wait-time (conf TOPOLOGY-MAX-REPLICATION-WAIT-TIME-SEC)
    -        total-wait-time (atom 0)
    -        current-replication-count (atom (if (:code-distributor nimbus) (.getReplicationCount (:code-distributor nimbus) storm-id) 0))]
    -  (if (:code-distributor nimbus)
    -    (while (and (> min-replication-count @current-replication-count)
    -             (or (= -1 max-replication-wait-time)
    -               (< @total-wait-time max-replication-wait-time)))
    +        current-replication-count-jar (if (not (local-mode? conf)) (atom
    +                                                                     (get-blob-replication-count (master-stormjar-key storm-id) nimbus))
    +                                                                     (atom min-replication-count))
    +        current-replication-count-code (atom (get-blob-replication-count (master-stormcode-key storm-id) nimbus))
    +        current-replication-count-conf (atom (get-blob-replication-count (master-stormconf-key storm-id) nimbus))
    +        total-wait-time (atom 0)]
    +    (log-message "wait for desired replication" "count"
    +      min-replication-count "wait-time" max-replication-wait-time
    +      "code" @current-replication-count-code
    +      "conf" @current-replication-count-conf
    +      "jar" @current-replication-count-jar
    +      "replication count" (get-blob-replication-count (master-stormconf-key storm-id) nimbus))
    +    (if (:blob-store nimbus)
    +      (while (and (> min-replication-count @current-replication-count-jar)
    +               (> min-replication-count @current-replication-count-code)
    +               (> min-replication-count @current-replication-count-conf)
    +               (or (= -1 max-replication-wait-time)
    +                 (< @total-wait-time max-replication-wait-time)))
             (sleep-secs 1)
             (log-debug "waiting for desired replication to be achieved.
               min-replication-count = " min-replication-count  " max-replication-wait-time = " max-replication-wait-time
    -          "current-replication-count = " @current-replication-count " total-wait-time " @total-wait-time)
    +          (if (not (local-mode? conf))"current-replication-count for jar key = " @current-replication-count-jar)
    +          "current-replication-count for code key = " @current-replication-count-code
    +          "current-replication-count for conf key = " @current-replication-count-conf
    +          " total-wait-time " @total-wait-time)
             (swap! total-wait-time inc)
    -        (reset! current-replication-count  (.getReplicationCount (:code-distributor nimbus) storm-id))))
    -  (if (< min-replication-count @current-replication-count)
    -    (log-message "desired replication count "  min-replication-count " achieved,
    -      current-replication-count" @current-replication-count)
    -    (log-message "desired replication count of "  min-replication-count " not achieved but we have hit the max wait time "
    -      max-replication-wait-time " so moving on with replication count = " @current-replication-count)
    -    )))
    -
    -(defn- read-storm-topology [conf storm-id]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -    (Utils/deserialize
    -      (FileUtils/readFileToByteArray
    -        (File. (master-stormcode-path stormroot))
    -        ) StormTopology)))
    +        (if (not (local-mode? conf))(reset! current-replication-count-conf  (get-blob-replication-count (master-stormconf-key storm-id))))
    +          (reset! current-replication-count-code  (get-blob-replication-count (master-stormcode-key storm-id)))
    +          (reset! current-replication-count-jar  (get-blob-replication-count (master-stormjar-key storm-id)))))
    +    (if (and (< min-replication-count @current-replication-count-conf)
    +          (< min-replication-count @current-replication-count-code)
    +          (< min-replication-count @current-replication-count-jar))
    +      (log-message "desired replication count of "  min-replication-count " not achieved but we have hit the max wait time "
    +        max-replication-wait-time " so moving on with replication count for conf key = " @current-replication-count-conf
    +        " for code key = " @current-replication-count-code "for jar key = " @current-replication-count-jar)
    +      (log-message "desired replication count "  min-replication-count " achieved,
    +        current-replication-count for conf key " @current-replication-count-conf ",
    +        current-replication-count for code key = " @current-replication-count-code ",
    +        current-replication-count for jar key = " @current-replication-count-jar))))
    --- End diff --
    
    What happens if we do not close the string literal after `" achieved,` and continue it on the next line with indentation? Does all of that white-space end up getting output?
    
    Also a nitpick: no `=` sign in `conf key "` while other two have one.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on the pull request:

    https://github.com/apache/storm/pull/845#issuecomment-158283901
  
    Utils.java
    
    * In general, the new methods we are adding here might be better off somewhere in the blobstore package.
    * Check indentation of changed code.  I see some indented at 2 spaces, some at 4.  Also, I see some inconsistent initial indentation.
    * Some of these methods are synchronized and hold a lock on Utils while they access ZK.  I can see this causing lots of problems.  Can we try and use more granular locking?  Some of this can be helped by moving the code to a more appropriate class or namespace.
    
    
    ```
    447     nconf.put(Config.BLOBSTORE_CLEANUP_ENABLE, new Boolean(true));
    ```
    I read it is dangerous to create instances of Boolean with given values, and is preferable to use `Boolean.TRUE` and `Boolean.FALSE`.
    
    
    ```
     452   // Meant to be called only by the supervisor for stormjar/stormconf/stormcode files.
     453   public static void downloadResourcesAsSupervisor(Map conf, String key, String localFile,
     454                                                    ClientBlobStore cb) throws AuthorizationExcep     tion, KeyNotFoundException, IOException {
     455     final int MAX_RETRY_ATTEMPTS = 2;
     456     final int ATTEMPTS_INTERVAL_TIME = 100;
     457     for (int retryAttempts = 0; retryAttempts < MAX_RETRY_ATTEMPTS; retryAttempts++) {
     458       if (downloadResourcesAsSupervisorAttempt(cb, key, localFile)) {
     459         break;
     460       }
     461       Utils.sleep(ATTEMPTS_INTERVAL_TIME);
     462     }
     463     //NO Exception on error the supervisor will try again after a while
     464   }
    ```
    
    * `conf` parameter not used.
    * The values here are hard-coded and are small.  What benefit does the sleep have?  Could we unroll the `for` loop?
    * If this is meant for the supervisor, does it belong in supervisor.clj?
    
    
    ```
     486     } finally {
     487       try {
     488         if (out != null) out.close();
     489       } catch (IOException ignored) {}
     490       try {
     491         if (in != null) in.close();
     492       } catch (IOException ignored) {}
     493     }
    ```
    
    * minor: braces for the `if` statements
    
    ```
     510     try {
     511       ReadableBlobMeta metadata = cb.getBlobMeta(key);
     512       nimbusBlobVersion = metadata.get_version();
     513     } catch (AuthorizationException | KeyNotFoundException exp) {
     514       throw exp;
     515     } catch (TException e) {
     516       throw new RuntimeException(e);
     517     }
    ```
    
    Can we ever catch a `TException` here?
    
    ```
     562   // only works on operating  systems that support posix
     563   public static void restrictPermissions(String baseDir) {
     564     try {
     565       Set<PosixFilePermission> perms = new HashSet<PosixFilePermission>(
     566               Arrays.asList(PosixFilePermission.OWNER_READ, PosixFilePermission.OWNER_WRITE,
     567                       PosixFilePermission.OWNER_EXECUTE, PosixFilePermission.GROUP_READ,
     568                       PosixFilePermission.GROUP_EXECUTE));
     569       Files.setPosixFilePermissions(FileSystems.getDefault().getPath(baseDir), perms);
     570     } catch (IOException e) {
     571       throw new RuntimeException(e);
     572     }
     573   }
    ```
    
    If this is only used by the supervisor, should we move it to supervisor.clj?
    
    ```
     627     public static byte[] thriftSerialize(TBase t) {
    
     641     public static <T> T thriftDeserialize(Class c, byte[] b, int offset, int length) {
     
     652     public static <T> T thriftDeserialize(Class c, byte[] b) {
    ```
    
    * `thriftDeserialize(Class c, byte[] b, int offset, int length)` does not appear to be used anywhere
    * The other two methods look nearly identical to code that is in Trident.  Could we make these common?
    
    ```
     756    * @param in InputStrem to read from
    ```
    
    `InputStream`
    
    
    ```
     928   // java equivalent of util.on-windows?
     929   public static boolean onWindows() {
     930     return System.getenv("OS") == "Windows_NT";
     931   }
    ```
    
    * Use `.equals` for String
    * Remove comment
    * Remove `util.on-windows?` and have clojure code call into this one.
    
    ```
     933   public static long unpack(File localrsrc, File dst) throws IOException {
    ```
    
    This always returns `0`, and the only place it's called ignores the return value.  We can make it `void`.
    
    ```
    1102      * Takes an input dir or file and returns the du on that local directory. Very basic
    ```
    
    `du` -> `disk usage`
    
    
    ```
    1375       if (isSuccess != true) {
    ```
    
    `!isSuccess`
    
    
    ```
    1405     public static synchronized void updateKeyForBlobStore (Map conf, BlobStore blobStore, CuratorFramework zkClient, String key, NimbusInfo nimbusDetails) {
    1406         try {
    1407           // This is to avoid mocking of unit tests at several places as nimbusDetails
    1408           // and moreover the nimbusInfo is not required for HdfsBlobstore only for LocalFsBlobstore
    1409             if (nimbusDetails == null) {
    1410                 return;
    1411             }
    1412             boolean isListContainsCurrentNimbusInfo = false;
    1413             List<String> stateInfo = new ArrayList<String>();
    ```
    
    * Check indentation throughout this method
    * Remove extra space before signature in declaration
    * If `nimbusDetails` is not required for HdfsBlobStore, why are we returning immediately if it is `null`?  Won't that break the HDFS case?
       * Could this be a member method of a BlobStore instead of a static method here in Utils?
    * No need to initialize `stateInfo`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45306963
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -811,6 +993,21 @@
           :else (-> value sub-fn (clojure.string/split #"\s+")))))
     
     
    +(defn create-blobstore-links
    +  "Create symlinks in worker launch directory for all blobs"
    +  [conf storm-id port worker-id]
    +  (let [stormroot (supervisor-stormdist-root conf storm-id)
    +        storm-conf (read-supervisor-storm-conf conf storm-id)
    +        workerroot (worker-root conf worker-id)
    +        blobstore-map (storm-conf TOPOLOGY-BLOBSTORE-MAP)
    +        blob-file-names (get-blob-file-names blobstore-map)
    +        resource-file-names (cons RESOURCES-SUBDIR blob-file-names)]
    +    (log-message "Creating symlinks for worker-id: " worker-id " storm-id: "
    +      storm-id " for files(" (count resource-file-names) "): " (pr-str resource-file-names))
    +    (create-symlink! workerroot stormroot RESOURCES-SUBDIR)
    +    (doseq [file-name blob-file-names]
    +      (create-symlink! workerroot stormroot file-name file-name))))
    --- End diff --
    
    removed the parameter


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by kishorvpatil <gi...@git.apache.org>.
Github user kishorvpatil commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43821851
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStore.java ---
    @@ -0,0 +1,278 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.*;
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.utils.Utils;
    +import backtype.storm.utils.ZookeeperAuthInfo;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.thrift.TBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.*;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +
    +import static backtype.storm.blobstore.BlobStoreAclHandler.*;
    +import static backtype.storm.utils.Utils.newCurator;
    +
    +/**
    + * Provides a local file system backed blob store implementation for Nimbus.
    + */
    +public class LocalFsBlobStore extends BlobStore {
    +  public static final Logger LOG = LoggerFactory.getLogger(LocalFsBlobStore.class);
    +  private static final String DATA_PREFIX = "data_";
    +  private static final String META_PREFIX = "meta_";
    +  protected BlobStoreAclHandler _aclHandler;
    +  private NimbusInfo nimbusInfo;
    +  private FileBlobStoreImpl fbs;
    +  Map conf;
    --- End diff --
    
    should this be private?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43952101
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -984,25 +1108,30 @@
             ))))
     
     (defn cleanup-corrupt-topologies! [nimbus]
    -  (if (is-leader nimbus :throw-exception false)
    -    (let [storm-cluster-state (:storm-cluster-state nimbus)
    -          code-ids (set (code-ids (:conf nimbus)))
    -          active-topologies (set (.active-storms storm-cluster-state))
    -          corrupt-topologies (set/difference active-topologies code-ids)]
    -      (doseq [corrupt corrupt-topologies]
    -        (log-message "Corrupt topology " corrupt " has state on zookeeper but doesn't have a local dir on Nimbus. Cleaning up...")
    -        (.remove-storm! storm-cluster-state corrupt)
    -        )))
    -  (log-message "not a leader, skipping cleanup-corrupt-topologies"))
    -
    -;;setsup code distributor entries for all current topologies for which code is available locally.
    -(defn setup-code-distributor [nimbus]
       (let [storm-cluster-state (:storm-cluster-state nimbus)
    -        locally-available-storm-ids (set (code-ids (:conf nimbus)))
    +        blob-store (:blob-store nimbus)
    +        code-ids (set (code-ids blob-store))
             active-topologies (set (.active-storms storm-cluster-state))
    -        locally-available-active-storm-ids (set/intersection locally-available-storm-ids active-topologies)]
    -    (doseq [storm-id locally-available-active-storm-ids]
    -      (.setup-code-distributor! storm-cluster-state storm-id (:nimbus-host-port-info nimbus)))))
    +        corrupt-topologies (set/difference active-topologies code-ids)]
    +    (doseq [corrupt corrupt-topologies]
    +      (log-message "Corrupt topology " corrupt " has state on zookeeper but doesn't have a local dir on Nimbus. Cleaning up...")
    +      (.remove-storm! storm-cluster-state corrupt)
    +      (if (instance? LocalFsBlobStore blob-store)
    +        (doseq [blob-key (get-key-list-from-id (:conf nimbus) corrupt)]
    +          (.remove-blobstore-key! storm-cluster-state blob-key)))
    +      )))
    +
    +;;setsup blobstore for all current keys
    --- End diff --
    
    remove comment or else make a docstring


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the pull request:

    https://github.com/apache/storm/pull/845#issuecomment-153511977
  
    @ptgoetz I don't consider this criticism and I'm not really trying to complain, I'm just trying to understand what requires an IP grant. 
    
    from http://incubator.apache.org/ip-clearance/index.html I find
    
    >  From time to time, an external codebase is brought into the ASF that is not a separate incubating project but still represents a substantial contribution that was not developed within the ASF's source control system and on our public mailing lists.
    
    and
    
    > The intent is to simply help to ensure, and record, that due diligence (Software Grant, CLA, Corp CLA, license and dependencies) has been paid to the incoming code, so that it can be merged into an existing project/module.
    
    Which seems a lot more vague than
    
    >   * Was the code developed in the open, using ASF infrastructure?
    >   * Was the Apache community aware of its existence from inception, or early on, such that it could contribute to it’s development?
    >   * Is the commit history intact such that all contributors can be identified?
    
    I can also see how these map from one to the other but not totally.  I am mostly confused about what is "a substantial contribution" which I am not disputing this is.  I am also confused about how putting a large pull request up differs from this?  Even though the core of this was developed in a separate repository anything and everything in this pull request is up for modification if the community feels that we need to.  The design document has been up for over a week, and the JIRA itself has been open since June that no one touched until now, but could have.  And most of the code has been touched in preparing it to work with Nimbus HA, specifically for this.  Lastly I don't see a reason to have the code contribution be attributed line by line, if we call out exactly who helped contribute to the patch.
    
    If you feel that we need an IP Grant for this I will make it happen, but going forward we the PMC need to have a discussion and clarify these questions, because to me leaving this up to a gut feeling is just going to put up too many barriers and potential frustration for developers.  Now I don't want line counts or anything like that, but something a lot less vague than "substantial"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by kishorvpatil <gi...@git.apache.org>.
Github user kishorvpatil commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43813101
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -1103,21 +1216,39 @@
          (.set_reset_log_level_timeout_epoch log-config (coerce/to-long timeout))
          (.unset_reset_log_level_timeout_epoch log-config))))
     
    +(defmethod blob-sync-code :distributed [conf nimbus]
    +  (if (not (is-leader nimbus :throw-exception false))
    +    (let [storm-cluster-state (:storm-cluster-state nimbus)
    +          nimbus-host-port-info (:nimbus-host-port-info nimbus)
    +          blob-store-key-list (get-key-list-from-blob-store (:blob-store nimbus))
    +          zk-key-list (into [] (.blobstore storm-cluster-state (fn [] (blob-sync-code conf nimbus))))]
    +      (log-debug "blob-sync-code " "keys " blob-store-key-list "blob-key " blob-store-key-list)
    +      (let [sync-object (doto
    +                          (SyncBlobs. (:blob-store nimbus) conf)
    +                          (.setNimbusInfo nimbus-host-port-info)
    +                          (.setBlobStoreKeyList (if (not-nil? blob-store-key-list) (java.util.ArrayList. blob-store-key-list) (java.util.ArrayList.)))
    +                          (.setZookeeperKeyList (if (not-nil? zk-key-list) (java.util.ArrayList. zk-key-list) (java.util.ArrayList.))))]
    --- End diff --
    
    Minor, but I think we should import java util ArrayList instead of repeating complete qualified name


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45133203
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -372,66 +389,50 @@
              ". State: " state
              ", Heartbeat: " (pr-str heartbeat))
             (shutdown-worker supervisor id)
    -        (if (:code-distributor supervisor)
    -          (.cleanup (:code-distributor supervisor) id))
    -        ))
    -
    -    (doseq [id (vals new-worker-ids)]
    -      (local-mkdirs (worker-pids-root conf id))
    -      (local-mkdirs (worker-heartbeats-root conf id)))
    -    (ls-approved-workers! local-state
    -          (merge
    -           (select-keys (ls-approved-workers local-state)
    -                        (keys keepers))
    -           (zipmap (vals new-worker-ids) (keys new-worker-ids))
    -           ))
    -
    -    ;; check storm topology code dir exists before launching workers
    -    (doseq [[port assignment] reassign-executors]
    -      (let [downloaded-storm-ids (set (read-downloaded-storm-ids conf))
    -            storm-id (:storm-id assignment)
    -            cached-assignment-info @(:assignment-versions supervisor)
    -            assignment-info (if (and (not-nil? cached-assignment-info) (contains? cached-assignment-info storm-id ))
    -                              (get cached-assignment-info storm-id)
    -                              (.assignment-info-with-version storm-cluster-state storm-id nil))
    -	    storm-code-map (read-storm-code-locations assignment-info)
    -            master-code-dir (if (contains? storm-code-map :data) (storm-code-map :data))
    -            stormroot (supervisor-stormdist-root conf storm-id)]
    -        (if-not (or (contains? downloaded-storm-ids storm-id) (.exists (File. stormroot)) (nil? master-code-dir))
    -          (download-storm-code conf storm-id master-code-dir supervisor download-lock))
             ))
    -
    -    (wait-for-workers-launch
    -     conf
    -     (dofor [[port assignment] reassign-executors]
    -            (let [id (new-worker-ids port)
    -                  storm-id (:storm-id assignment)
    -                  ^WorkerResources resources (:resources assignment)
    -                  mem-onheap (.get_mem_on_heap resources)]
    -              (try
    -                (log-message "Launching worker with assignment "
    -                             (pr-str assignment)
    -                             " for this supervisor "
    -                             (:supervisor-id supervisor)
    -                             " on port "
    -                             port
    -                             " with id "
    -                             id
    -                             )
    -                (launch-worker supervisor
    -                               (:storm-id assignment)
    -                               port
    -                               id
    -                               mem-onheap)
    -                (mark! supervisor:num-workers-launched)
    -                (catch java.io.FileNotFoundException e
    -                  (log-message "Unable to launch worker due to "
    -                               (.getMessage e)))
    -                (catch java.io.IOException e
    -                  (log-message "Unable to launch worker due to "
    -                               (.getMessage e))))
    -         id)))
    -    ))
    +    (let [valid-new-worker-ids
    +          (into {}
    +            (remove nil?
    +              (dofor [[port assignment] reassign-executors]
    +                (let [id (new-worker-ids port)
    +                      storm-id (:storm-id assignment)
    +                      ^WorkerResources resources (:resources assignment)
    +                      mem-onheap (.get_mem_on_heap resources)]
    +                  ;; This condition checks for required files exist before launching the worker
    +                  (if (required-topo-files-exist? conf storm-id)
    +                    (do
    +                      (log-message "Launching worker with assignment "
    +                        (pr-str assignment)
    +                        " for this supervisor "
    +                        (:supervisor-id supervisor)
    +                        " on port "
    +                        port
    +                        " with id "
    +                        id)
    +                      (local-mkdirs (worker-pids-root conf id))
    +                      (local-mkdirs (worker-heartbeats-root conf id))
    +                      (launch-worker supervisor
    +                        (:storm-id assignment)
    +                        port
    +                        id
    +                        mem-onheap)
    +                      [port id])
    --- End diff --
    
    Also, the resulting name of the bound var could be something like `id->port` or `valid-new-id->port`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r44996296
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -454,10 +456,78 @@
           (shutdown-worker supervisor id))
         ))
     
    +(defn get-blob-localname
    +  "Given the blob information either gets the localname field if it exists,
    +  else routines the default value passed in."
    +  [blob-info defaultValue]
    +  (if-let [val (if blob-info (get blob-info "localname") nil)] val defaultValue))
    +
    +(defn should-uncompress-blob?
    +  "Given the blob information returns the value of the uncompress field, handling it either being
    +  a string or a boolean value, or ifs its not specified then returns false"
    +  [blob-info]
    +  (boolean (and blob-info
    +             (if-let [val (get blob-info "uncompress")]
    +               (.booleanValue (Boolean. val))))))
    +
    +(defn remove-blob-references
    +  "Remove a reference to a blob when its no longer needed."
    +  [localizer storm-id conf]
    +  (let [storm-conf (read-supervisor-storm-conf conf storm-id)
    +        blobstore-map (storm-conf TOPOLOGY-BLOBSTORE-MAP)
    +        user (storm-conf TOPOLOGY-SUBMITTER-USER)
    +        topo-name (storm-conf TOPOLOGY-NAME)]
    +    (if blobstore-map (doseq [[k, v] blobstore-map]
    --- End diff --
    
    the 'then' part should be on a new line.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43808124
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -346,53 +387,104 @@
           [(.getNodeId slot) (.getPort slot)]
           )))
     
    +(defn- get-metadata-version [blob-store key subject]
    +  (let [blob-meta (.getBlobMeta blob-store key subject)]
    +    (.get_version blob-meta)))
    +
    +(defn get-key-list-from-blob-store [blob-store]
    +  (let [key-iter (.listKeys blob-store (get-nimbus-subject))]
    +    (into [] (iterator-seq key-iter))))
    +
     (defn- setup-storm-code [nimbus conf storm-id tmp-jar-location storm-conf topology]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -   (log-message "nimbus file location:" stormroot)
    -   (FileUtils/forceMkdir (File. stormroot))
    -   (FileUtils/cleanDirectory (File. stormroot))
    -   (setup-jar conf tmp-jar-location stormroot)
    -   (FileUtils/writeByteArrayToFile (File. (master-stormcode-path stormroot)) (Utils/serialize topology))
    -   (FileUtils/writeByteArrayToFile (File. (master-stormconf-path stormroot)) (Utils/toCompressedJsonConf storm-conf))
    -   (if (:code-distributor nimbus) (.upload (:code-distributor nimbus) stormroot storm-id))
    -   ))
    +  (let [subject (get-subject)
    +        storm-cluster-state (:storm-cluster-state nimbus)
    +        blob-store (:blob-store nimbus)
    +        jar-key (master-stormjar-key storm-id)
    +        code-key (master-stormcode-key storm-id)
    +        conf-key (master-stormconf-key storm-id)
    +        nimbus-host-port-info (:nimbus-host-port-info nimbus)]
    +    (if tmp-jar-location ;;in local mode there is no jar
    +      (do
    +        (.createBlob blob-store jar-key (FileInputStream. tmp-jar-location) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +        (if (instance? LocalFsBlobStore blob-store)
    +          (.setup-blobstore! storm-cluster-state jar-key nimbus-host-port-info (get-metadata-version blob-store jar-key subject)))))
    +    (.createBlob blob-store conf-key (Utils/toCompressedJsonConf storm-conf) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state conf-key nimbus-host-port-info (get-metadata-version blob-store conf-key subject)))
    +    (.createBlob blob-store code-key (Utils/serialize topology) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state code-key nimbus-host-port-info (get-metadata-version blob-store code-key subject)))))
    +
    +(defn- read-storm-topology [storm-id blob-store]
    +  (Utils/deserialize
    +    (.readBlob blob-store (master-stormcode-key storm-id) (get-subject)) StormTopology))
    +
    +(defn- get-blob-replication-count [blob-key nimbus]
    +  (if (:blob-store nimbus)
    +          (-> (:blob-store nimbus)
    +            (.getBlobReplication  blob-key (get-nimbus-subject))
    +            (.get_replication))))
     
     (defn- wait-for-desired-code-replication [nimbus conf storm-id]
       (let [min-replication-count (conf TOPOLOGY-MIN-REPLICATION-COUNT)
             max-replication-wait-time (conf TOPOLOGY-MAX-REPLICATION-WAIT-TIME-SEC)
    -        total-wait-time (atom 0)
    -        current-replication-count (atom (if (:code-distributor nimbus) (.getReplicationCount (:code-distributor nimbus) storm-id) 0))]
    -  (if (:code-distributor nimbus)
    -    (while (and (> min-replication-count @current-replication-count)
    +        current-replication-count-jar (if (not (local-mode? conf)) (atom
    +                                                                     (get-blob-replication-count (master-stormjar-key storm-id) nimbus))
    +                                                                     (atom min-replication-count))
    +        current-replication-count-code (atom (get-blob-replication-count (master-stormcode-key storm-id) nimbus))
    +        current-replication-count-conf (atom (get-blob-replication-count (master-stormconf-key storm-id) nimbus))
    +        total-wait-time (atom 0)]
    +    (log-message "wait for desired replication" "count"
    +      min-replication-count "wait-time" max-replication-wait-time
    +      "code" @current-replication-count-code
    +      "conf" @current-replication-count-conf
    +      "jar" @current-replication-count-jar
    +      "replication count" (get-blob-replication-count (master-stormconf-key storm-id) nimbus))
    +  (if (:blob-store nimbus)
    +    (while (and (> min-replication-count @current-replication-count-jar)
    +                (> min-replication-count @current-replication-count-code)
    +                (> min-replication-count @current-replication-count-conf)
                  (or (= -1 max-replication-wait-time)
    --- End diff --
    
    Indentation or incorrect parenthesis?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on the pull request:

    https://github.com/apache/storm/pull/845#issuecomment-153478106
  
    storm.thrift:
    
    ```
    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
    }
    ```
    
    Really this should be a `union` of two types: `OtherAccessControl`, which lacks a `name`, and `UserAccessControl`, which requires `name`.
    
    It's invalid to have `type == AccessControlType.OTHER` and with a `name`, or to have `type == AccessControlType.USER` without a `name`.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45427932
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/KeyVersion.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * 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.zookeeper.CreateMode;
    +import org.apache.zookeeper.ZooDefs;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.TreeSet;
    +import java.util.Map;
    +import java.util.List;
    +
    +/**
    + * Class hands over the version of the key to be stored within the zookeeper
    + */
    +public class KeyVersion {
    +  private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
    +  private final String BLOBSTORE_SUBTREE="/blobstore";
    +  private final String BLOBSTORE_KEY_COUNTER_SUBTREE="/blobstorekeycounter";
    +  private String key;
    +  private NimbusInfo nimbusInfo;
    +
    +  public KeyVersion(String key, NimbusInfo nimbusInfo) {
    +    this.key = key;
    +    this.nimbusInfo = nimbusInfo;
    +  }
    +
    +  public int getKeyVersion(Map conf) {
    +    TreeSet<Integer> versions = new TreeSet<Integer>();
    +    CuratorFramework zkClient = Utils.createZKClient(conf);
    +    try {
    +      // Key has not been created yet and it is the first time it is being created
    +      if(zkClient.checkExists().forPath(BLOBSTORE_SUBTREE + "/" + key) == null) {
    +        zkClient.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT)
    +                .withACL(ZooDefs.Ids.OPEN_ACL_UNSAFE).forPath(BLOBSTORE_KEY_COUNTER_SUBTREE + "/" + key + "/" + 1);
    +        return 1;
    +      }
    +
    +      // When all nimbodes go down and one or few of them come up
    +      // Unfortunately there might not be an exact way to know which one contains the most updated blob
    +      // if all go down which is unlikely. Hence there might be a need to update the blob if all go down
    +      List<String> stateInfoList = zkClient.getChildren().forPath(BLOBSTORE_SUBTREE + "/" + key);
    +      LOG.debug("stateInfoList {} stateInfoList {}", stateInfoList.size(), stateInfoList);
    +      if(stateInfoList.isEmpty()) {
    +        return getKeyVersionCounterValue(zkClient, key);
    +      }
    +
    +      LOG.debug("stateInfoSize {}", stateInfoList.size());
    +      // In all other cases check for the latest version on the nimbus and assign the version
    +      // check if all are have same version, if not assign the highest version
    +      for (String stateInfo:stateInfoList) {
    +        versions.add(Integer.parseInt(Utils.normalizeVersionInfo(stateInfo)[1]));
    +      }
    +
    +      int currentCounter = getKeyVersionCounterValue(zkClient, key);
    +      // This condition returns version when a nimbus crashes and comes up
    +      if (!checkIfStateContainsCurrentNimbusHost(stateInfoList, nimbusInfo) && !nimbusInfo.isLeader()) {
    +        if (versions.last() < currentCounter) {
    +          return currentCounter;
    +        } else {
    +          return currentCounter - 1;
    +        }
    +      }
    +      // Condition checks for an update scenario
    +      if (stateInfoList.size() >= 1 && versions.size() == 1) {
    +        if (versions.first() < getKeyVersionCounterValue(zkClient, key)) {
    +          incrementCounter(zkClient, key, currentCounter);
    +          return currentCounter + 1;
    +        } else {
    +          incrementCounter(zkClient, key, currentCounter);
    +          return versions.first() + 1;
    +        }
    +      }
    +    } catch(Exception e) {
    +      LOG.error("Exception {}", e);
    +    } finally {
    +      if (zkClient != null) {
    +        zkClient.close();
    +      }
    +    }
    +    return versions.last();
    +  }
    +
    +  public boolean checkIfStateContainsCurrentNimbusHost(List<String> stateInfoList, NimbusInfo nimbusInfo) {
    +    boolean containsNimbusHost = false;
    +    for(String stateInfo:stateInfoList) {
    +      if(stateInfo.contains(nimbusInfo.getHost())) {
    +        containsNimbusHost = true;
    +        break;
    +      }
    +    }
    +    return containsNimbusHost;
    +  }
    +
    +  public void incrementCounter(CuratorFramework zkClient, String key, int count) throws Exception {
    --- End diff --
    
    `private`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by ptgoetz <gi...@git.apache.org>.
Github user ptgoetz commented on the pull request:

    https://github.com/apache/storm/pull/845#issuecomment-153497694
  
    @revans2 
    
    >I know you want to be careful about what gets merged in, and if you insist we will go through the IP clearance process, but if we can avoid it I really would prefer to do so. We have not had any issues in the past with large commits, like when we contributed the security code to storm which followed essentially the same process as this code. On my end it is probably going to take longer than the 72 hour waiting period to track down the right people to get signatures and they will all ask me why we have to do this when my team has blanket approval to contribute to storm.
    
    >Like I said if you insist on getting IP clearance we will do it, but all it is just going to do is add more pain for me which I really would like to avoid if I can.
    
    My apologies for seeming to come out of the blue on this, it’s not my intention to put up roadblocks or delay anything. Rather, I’m trying to make sure _**we, the PMC**_ do a better job of complying with ASF policy. It’s become clear to me that there were a number of cases in the past where we did not properly adhere to the IP Clearance policy. A lot of that stems from a lack of knowledge on my part and others’ as to when IP Clearance is required. I’m trying to correct that going forward.
    
    Admittedly, the paragraph regarding IP Clearance is vague. Here are a few points about the pull request that triggered the “IP Clearance flag” in my head (please correct me if I’m wrong about anything):
    
    * Was the code developed in the open, using ASF infrastructure? (No, it was developed internally at Yahoo, even though the intent was to eventually contribute it to the community).
    * Was the Apache community aware of its existence from inception, or early on, such that it could contribute to it’s development? (No, see previous point. Also, the initial JIRA stated that it was under development internally at Yahoo.)
    * Is the commit history intact such that all contributors can be identified? (No, all commits are by one individual, and the initial commit appears to be an import of an existing codebase that might have involved other contributors.)
    
    Please don’t take any of the above as criticisms, I’m just pointing out what triggered IP clearance consideration in my head. 
    
    If all the contributors involved are covered either by a CCLA or an ICLA, I think all we would need is a software grant. I can do the rest of the work, and the code review can certainly continue.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r44989413
  
    --- Diff: external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java ---
    @@ -0,0 +1,381 @@
    +/**
    + * 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 org.apache.storm.hdfs.blobstore;
    +
    +import backtype.storm.Config;
    +import backtype.storm.blobstore.AtomicOutputStream;
    +import backtype.storm.blobstore.AtomicOutputStream;
    +import backtype.storm.blobstore.BlobStore;
    +import backtype.storm.blobstore.BlobStoreAclHandler;
    +import backtype.storm.blobstore.BlobStoreFile;
    +import backtype.storm.blobstore.InputStreamWithMeta;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.utils.NimbusClient;
    +import backtype.storm.utils.Utils;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.security.UserGroupInformation;
    +import org.apache.thrift7.TBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.ByteArrayOutputStream;
    +import java.io.FileNotFoundException;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.security.AccessController;
    +import java.security.PrivilegedAction;
    +import java.util.Iterator;
    +import java.util.Map;
    +
    +import static backtype.storm.blobstore.BlobStoreAclHandler.ADMIN;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.READ;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.WRITE;
    +
    +/**
    + * Provides a HDFS file system backed blob store implementation.
    + * Note that this provides an api for having HDFS be the backing store for the blobstore,
    + * it is not a service/daemon.
    + */
    +public class HdfsBlobStore extends BlobStore {
    --- End diff --
    
    This file uses 2-space indentation, whereas the rest of the project uses 4-space.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by unsleepy22 <gi...@git.apache.org>.
Github user unsleepy22 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46024124
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStore.java ---
    @@ -0,0 +1,296 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.utils.Utils;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.ByteArrayOutputStream;
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.FileNotFoundException;
    +import java.io.InputStream;
    +
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;;
    +
    +import static backtype.storm.blobstore.BlobStoreAclHandler.ADMIN;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.READ;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.WRITE;
    +
    +/**
    + * Provides a local file system backed blob store implementation for Nimbus.
    + */
    +public class LocalFsBlobStore extends BlobStore {
    +  public static final Logger LOG = LoggerFactory.getLogger(LocalFsBlobStore.class);
    +  private static final String DATA_PREFIX = "data_";
    +  private static final String META_PREFIX = "meta_";
    +  protected BlobStoreAclHandler _aclHandler;
    +  private final String BLOBSTORE_SUBTREE = "/blobstore/";
    +  private NimbusInfo nimbusInfo;
    +  private FileBlobStoreImpl fbs;
    +  private Map conf;
    +
    +  @Override
    +  public void prepare(Map conf, String overrideBase, NimbusInfo nimbusInfo) {
    +    this.conf = conf;
    +    this.nimbusInfo = nimbusInfo;
    +     if (overrideBase == null) {
    +       overrideBase = (String)conf.get(Config.BLOBSTORE_DIR);
    +      if (overrideBase == null) {
    +        overrideBase = (String) conf.get(Config.STORM_LOCAL_DIR);
    +      }
    +    }
    +    File baseDir = new File(overrideBase, BASE_BLOBS_DIR_NAME);
    +    try {
    +      fbs = new FileBlobStoreImpl(baseDir, conf);
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +    _aclHandler = new BlobStoreAclHandler(conf);
    +  }
    +
    +  @Override
    +  public AtomicOutputStream createBlob(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException {
    +    LOG.debug("Creating Blob for key {}", key);
    +    validateKey(key);
    +    _aclHandler.normalizeSettableBlobMeta(key, meta, who, READ | WRITE | ADMIN);
    +    BlobStoreAclHandler.validateSettableACLs(key, meta.get_acl());
    +    _aclHandler.hasPermissions(meta.get_acl(), READ | WRITE | ADMIN, who, key);
    +    if (fbs.exists(DATA_PREFIX+key)) {
    +      throw new KeyAlreadyExistsException(key);
    +    }
    +    BlobStoreFileOutputStream mOut = null;
    +    try {
    +      mOut = new BlobStoreFileOutputStream(fbs.write(META_PREFIX+key, true));
    +      mOut.write(Utils.thriftSerialize(meta));
    +      mOut.close();
    +      mOut = null;
    +      return new BlobStoreFileOutputStream(fbs.write(DATA_PREFIX+key, true));
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    } finally {
    +      if (mOut != null) {
    +        try {
    +          mOut.cancel();
    +        } catch (IOException e) {
    +          //Ignored
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public AtomicOutputStream updateBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    validateKey(key);
    +    checkForBlobOrDownload(key);
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(meta.get_acl(), WRITE, who, key);
    +    try {
    +      return new BlobStoreFileOutputStream(fbs.write(DATA_PREFIX+key, false));
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  private SettableBlobMeta getStoredBlobMeta(String key) throws KeyNotFoundException {
    +    InputStream in = null;
    +    try {
    +      LocalFsBlobStoreFile pf = fbs.read(META_PREFIX+key);
    +      try {
    +        in = pf.getInputStream();
    +      } catch (FileNotFoundException fnf) {
    +        throw new KeyNotFoundException(key);
    +      }
    +      ByteArrayOutputStream out = new ByteArrayOutputStream();
    +      byte [] buffer = new byte[2048];
    +      int len;
    +      while ((len = in.read(buffer)) > 0) {
    +        out.write(buffer, 0, len);
    +      }
    +      in.close();
    +      in = null;
    +      return Utils.thriftDeserialize(SettableBlobMeta.class, out.toByteArray());
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    } finally {
    +      if (in != null) {
    --- End diff --
    
    add a helper method like `closeStreamIfNotNull` to address all similar code blocks? I see a dozen of similar blocks...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45132995
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -372,66 +389,50 @@
              ". State: " state
              ", Heartbeat: " (pr-str heartbeat))
             (shutdown-worker supervisor id)
    -        (if (:code-distributor supervisor)
    -          (.cleanup (:code-distributor supervisor) id))
    -        ))
    -
    -    (doseq [id (vals new-worker-ids)]
    -      (local-mkdirs (worker-pids-root conf id))
    -      (local-mkdirs (worker-heartbeats-root conf id)))
    -    (ls-approved-workers! local-state
    -          (merge
    -           (select-keys (ls-approved-workers local-state)
    -                        (keys keepers))
    -           (zipmap (vals new-worker-ids) (keys new-worker-ids))
    -           ))
    -
    -    ;; check storm topology code dir exists before launching workers
    -    (doseq [[port assignment] reassign-executors]
    -      (let [downloaded-storm-ids (set (read-downloaded-storm-ids conf))
    -            storm-id (:storm-id assignment)
    -            cached-assignment-info @(:assignment-versions supervisor)
    -            assignment-info (if (and (not-nil? cached-assignment-info) (contains? cached-assignment-info storm-id ))
    -                              (get cached-assignment-info storm-id)
    -                              (.assignment-info-with-version storm-cluster-state storm-id nil))
    -	    storm-code-map (read-storm-code-locations assignment-info)
    -            master-code-dir (if (contains? storm-code-map :data) (storm-code-map :data))
    -            stormroot (supervisor-stormdist-root conf storm-id)]
    -        (if-not (or (contains? downloaded-storm-ids storm-id) (.exists (File. stormroot)) (nil? master-code-dir))
    -          (download-storm-code conf storm-id master-code-dir supervisor download-lock))
             ))
    -
    -    (wait-for-workers-launch
    -     conf
    -     (dofor [[port assignment] reassign-executors]
    -            (let [id (new-worker-ids port)
    -                  storm-id (:storm-id assignment)
    -                  ^WorkerResources resources (:resources assignment)
    -                  mem-onheap (.get_mem_on_heap resources)]
    -              (try
    -                (log-message "Launching worker with assignment "
    -                             (pr-str assignment)
    -                             " for this supervisor "
    -                             (:supervisor-id supervisor)
    -                             " on port "
    -                             port
    -                             " with id "
    -                             id
    -                             )
    -                (launch-worker supervisor
    -                               (:storm-id assignment)
    -                               port
    -                               id
    -                               mem-onheap)
    -                (mark! supervisor:num-workers-launched)
    -                (catch java.io.FileNotFoundException e
    -                  (log-message "Unable to launch worker due to "
    -                               (.getMessage e)))
    -                (catch java.io.IOException e
    -                  (log-message "Unable to launch worker due to "
    -                               (.getMessage e))))
    -         id)))
    -    ))
    +    (let [valid-new-worker-ids
    +          (into {}
    +            (remove nil?
    +              (dofor [[port assignment] reassign-executors]
    +                (let [id (new-worker-ids port)
    +                      storm-id (:storm-id assignment)
    +                      ^WorkerResources resources (:resources assignment)
    +                      mem-onheap (.get_mem_on_heap resources)]
    +                  ;; This condition checks for required files exist before launching the worker
    +                  (if (required-topo-files-exist? conf storm-id)
    +                    (do
    +                      (log-message "Launching worker with assignment "
    +                        (pr-str assignment)
    +                        " for this supervisor "
    +                        (:supervisor-id supervisor)
    +                        " on port "
    +                        port
    +                        " with id "
    +                        id)
    +                      (local-mkdirs (worker-pids-root conf id))
    +                      (local-mkdirs (worker-heartbeats-root conf id))
    +                      (launch-worker supervisor
    +                        (:storm-id assignment)
    +                        port
    +                        id
    +                        mem-onheap)
    +                      [port id])
    --- End diff --
    
    If we return `[id port]` here instead of `[port id]` here, we do not have to reverse the map later on when we use it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43780269
  
    --- Diff: storm-core/src/clj/backtype/storm/blobstore.clj ---
    @@ -0,0 +1,28 @@
    +;; 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.
    +
    +(ns backtype.storm.blobstore
    +  (:import [backtype.storm.utils Utils])
    +  (:import [backtype.storm.blobstore ClientBlobStore])
    +  (:use [backtype.storm config]))
    +
    +(defmacro with-configured-blob-client
    +  [client-sym & body]
    +  `(let [conf# (read-storm-config)
    +         ^ClientBlobStore ~client-sym (Utils/getClientBlobStore conf#)]
    +    (try
    +      ~@body
    +    (finally (.shutdown ~client-sym)))))
    --- End diff --
    
    And here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45261961
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -476,14 +539,21 @@
                                                (:sync-retry supervisor))
               new-assignment (->> all-assignment
                                   (filter-key #(.confirmAssigned isupervisor %)))
    -          assigned-storm-ids (assigned-storm-ids-from-port-assignments new-assignment)]
    +          rm-blob-refs? true
    --- End diff --
    
    This isn't a predicate (function that returns a boolean), it's just a var holding `true`.  We only use it in one spot, so maybe we should remove this binding and pass `true` below?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by unsleepy22 <gi...@git.apache.org>.
Github user unsleepy22 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46017732
  
    --- Diff: external/storm-hdfs/pom.xml ---
    @@ -103,6 +114,65 @@
                 </exclusions>
             </dependency>
             <dependency>
    +            <groupId>org.apache.hadoop</groupId>
    +            <artifactId>hadoop-auth</artifactId>
    +            <version>2.4.0</version>
    +            <exclusions>
    +                <exclusion>
    +                    <groupId>junit</groupId>
    +                    <artifactId>junit</artifactId>
    +                </exclusion>
    +                <exclusion>
    +                    <groupId>org.mockito</groupId>
    +                    <artifactId>mockito-all</artifactId>
    +                </exclusion>
    +                <exclusion>
    +                    <groupId>org.mortbay.jetty</groupId>
    +                    <artifactId>jetty-util</artifactId>
    +                </exclusion>
    +                <exclusion>
    +                    <groupId>org.mortbay.jetty</groupId>
    +                    <artifactId>jetty</artifactId>
    +                </exclusion>
    +                <exclusion>
    +                    <groupId>javax.servlet</groupId>
    +                    <artifactId>servlet-api</artifactId>
    +                </exclusion>
    +                <exclusion>
    +                    <groupId>org.slf4j</groupId>
    +                    <artifactId>slf4j-api</artifactId>
    +                </exclusion>
    +                <exclusion>
    +                    <groupId>commons-codec</groupId>
    +                    <artifactId>commons-codec</artifactId>
    +                </exclusion>
    +                <exclusion>
    +                    <groupId>log4j</groupId>
    +                    <artifactId>log4j</artifactId>
    +                </exclusion>
    +                <exclusion>
    +                    <groupId>org.slf4j</groupId>
    --- End diff --
    
    I see lots of exclusion of slf4j-log4j12, will it be better to add a direct slf4j-log4j12 dependency and set its scope to be provided?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r44987127
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -1047,25 +1162,34 @@
           (ls-topo-hist! topo-history-state new-history))))
     
     (defn cleanup-corrupt-topologies! [nimbus]
    -  (if (is-leader nimbus :throw-exception false)
    -    (let [storm-cluster-state (:storm-cluster-state nimbus)
    -          code-ids (set (code-ids (:conf nimbus)))
    -          active-topologies (set (.active-storms storm-cluster-state))
    -          corrupt-topologies (set/difference active-topologies code-ids)]
    -      (doseq [corrupt corrupt-topologies]
    -        (log-message "Corrupt topology " corrupt " has state on zookeeper but doesn't have a local dir on Nimbus. Cleaning up...")
    -        (.remove-storm! storm-cluster-state corrupt)
    -        )))
    -  (log-message "not a leader, skipping cleanup-corrupt-topologies"))
    -
    -;;setsup code distributor entries for all current topologies for which code is available locally.
    -(defn setup-code-distributor [nimbus]
       (let [storm-cluster-state (:storm-cluster-state nimbus)
    -        locally-available-storm-ids (set (code-ids (:conf nimbus)))
    +        blob-store (:blob-store nimbus)
    +        code-ids (set (code-ids blob-store))
             active-topologies (set (.active-storms storm-cluster-state))
    -        locally-available-active-storm-ids (set/intersection locally-available-storm-ids active-topologies)]
    -    (doseq [storm-id locally-available-active-storm-ids]
    -      (.setup-code-distributor! storm-cluster-state storm-id (:nimbus-host-port-info nimbus)))))
    +        corrupt-topologies (set/difference active-topologies code-ids)]
    +    (doseq [corrupt corrupt-topologies]
    +      (log-message "Corrupt topology " corrupt " has state on zookeeper but doesn't have a local dir on Nimbus. Cleaning up...")
    +      (.remove-storm! storm-cluster-state corrupt)
    +      (if (instance? LocalFsBlobStore blob-store)
    +        (doseq [blob-key (get-key-list-from-id (:conf nimbus) corrupt)]
    +          (.remove-blobstore-key! storm-cluster-state blob-key))))))
    +
    +;;setsup blobstore for all current keys
    --- End diff --
    
    We can make a docstring out of this comment.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45124681
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -34,6 +39,7 @@
                 [clojure.set :as set])
       (:import [org.apache.zookeeper data.ACL ZooDefs$Ids ZooDefs$Perms])
       (:import [org.yaml.snakeyaml Yaml]
    +           [java.util Date]
    --- End diff --
    
    Unused?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on the pull request:

    https://github.com/apache/storm/pull/845#issuecomment-158974561
  
    @d2r 
    396 struct ReadableBlobMeta {
    397   1: required SettableBlobMeta settable;
    398   //This is some indication of a version of a BLOB.  The only guarantee is
    399   // if the data changed in the blob the version will be different.
    400   2: required i64 version;
    401 }
    
    This blob version depends on the modified timestamp of the blob. The versions or sequence numbers I hand over for nimbus HA are different as I have written a KeyVersion class to hand over the version details for synchronizing various blobs on several nimbodes


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46099843
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStore.java ---
    @@ -0,0 +1,296 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.utils.Utils;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.ByteArrayOutputStream;
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.FileNotFoundException;
    +import java.io.InputStream;
    +
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;;
    +
    +import static backtype.storm.blobstore.BlobStoreAclHandler.ADMIN;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.READ;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.WRITE;
    +
    +/**
    + * Provides a local file system backed blob store implementation for Nimbus.
    + */
    +public class LocalFsBlobStore extends BlobStore {
    +  public static final Logger LOG = LoggerFactory.getLogger(LocalFsBlobStore.class);
    +  private static final String DATA_PREFIX = "data_";
    +  private static final String META_PREFIX = "meta_";
    +  protected BlobStoreAclHandler _aclHandler;
    +  private final String BLOBSTORE_SUBTREE = "/blobstore/";
    +  private NimbusInfo nimbusInfo;
    +  private FileBlobStoreImpl fbs;
    +  private Map conf;
    +
    +  @Override
    +  public void prepare(Map conf, String overrideBase, NimbusInfo nimbusInfo) {
    +    this.conf = conf;
    +    this.nimbusInfo = nimbusInfo;
    +     if (overrideBase == null) {
    +       overrideBase = (String)conf.get(Config.BLOBSTORE_DIR);
    +      if (overrideBase == null) {
    +        overrideBase = (String) conf.get(Config.STORM_LOCAL_DIR);
    +      }
    +    }
    +    File baseDir = new File(overrideBase, BASE_BLOBS_DIR_NAME);
    +    try {
    +      fbs = new FileBlobStoreImpl(baseDir, conf);
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +    _aclHandler = new BlobStoreAclHandler(conf);
    +  }
    +
    +  @Override
    +  public AtomicOutputStream createBlob(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException {
    +    LOG.debug("Creating Blob for key {}", key);
    +    validateKey(key);
    +    _aclHandler.normalizeSettableBlobMeta(key, meta, who, READ | WRITE | ADMIN);
    +    BlobStoreAclHandler.validateSettableACLs(key, meta.get_acl());
    +    _aclHandler.hasPermissions(meta.get_acl(), READ | WRITE | ADMIN, who, key);
    +    if (fbs.exists(DATA_PREFIX+key)) {
    +      throw new KeyAlreadyExistsException(key);
    +    }
    +    BlobStoreFileOutputStream mOut = null;
    +    try {
    +      mOut = new BlobStoreFileOutputStream(fbs.write(META_PREFIX+key, true));
    +      mOut.write(Utils.thriftSerialize(meta));
    +      mOut.close();
    +      mOut = null;
    +      return new BlobStoreFileOutputStream(fbs.write(DATA_PREFIX+key, true));
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    } finally {
    +      if (mOut != null) {
    +        try {
    +          mOut.cancel();
    +        } catch (IOException e) {
    +          //Ignored
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public AtomicOutputStream updateBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    validateKey(key);
    +    checkForBlobOrDownload(key);
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(meta.get_acl(), WRITE, who, key);
    +    try {
    +      return new BlobStoreFileOutputStream(fbs.write(DATA_PREFIX+key, false));
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  private SettableBlobMeta getStoredBlobMeta(String key) throws KeyNotFoundException {
    +    InputStream in = null;
    +    try {
    +      LocalFsBlobStoreFile pf = fbs.read(META_PREFIX+key);
    +      try {
    +        in = pf.getInputStream();
    +      } catch (FileNotFoundException fnf) {
    +        throw new KeyNotFoundException(key);
    +      }
    +      ByteArrayOutputStream out = new ByteArrayOutputStream();
    +      byte [] buffer = new byte[2048];
    +      int len;
    +      while ((len = in.read(buffer)) > 0) {
    +        out.write(buffer, 0, len);
    +      }
    +      in.close();
    +      in = null;
    +      return Utils.thriftDeserialize(SettableBlobMeta.class, out.toByteArray());
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    } finally {
    +      if (in != null) {
    --- End diff --
    
    I thought this is the only place we use in LocalFsBlobStore


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46338107
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -728,29 +848,92 @@
       (.shutdown supervisor)
       )
     
    -(defn setup-storm-code-dir [conf storm-conf dir]
    +(defn setup-storm-code-dir
    +  [conf storm-conf dir]
      (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
       (worker-launcher-and-wait conf (storm-conf TOPOLOGY-SUBMITTER-USER) ["code-dir" dir] :log-prefix (str "setup conf for " dir))))
     
    +(defn setup-blob-permission
    +  [conf storm-conf path]
    +  (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
    +    (worker-launcher-and-wait conf (storm-conf TOPOLOGY-SUBMITTER-USER) ["blob" path] :log-prefix (str "setup blob permissions for " path))))
    +
    +(defn download-blobs-for-topology!
    +  "Download all blobs listed in the topology configuration for a given topology."
    +  [conf stormconf-path localizer tmproot]
    +  (let [storm-conf (read-supervisor-storm-conf-given-path conf stormconf-path)
    +        blobstore-map (storm-conf TOPOLOGY-BLOBSTORE-MAP)
    +        user (storm-conf TOPOLOGY-SUBMITTER-USER)
    +        topo-name (storm-conf TOPOLOGY-NAME)
    +        user-dir (.getLocalUserFileCacheDir localizer user)
    +        localresources (blobstore-map-to-localresources blobstore-map)]
    +    (when localresources
    +      (when-not (.exists user-dir)
    +        (FileUtils/forceMkdir user-dir)
    +        (setup-blob-permission conf storm-conf (.toString user-dir)))
    --- End diff --
    
    Minor, but I think we do not need to `setup-blob-permission` here, since we are doing it next unconditionally.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45760606
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/IBlobWatcher.java ---
    @@ -0,0 +1,25 @@
    +/**
    + * 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;
    +
    +/**
    + * Provides a way to know when a blob changes.
    + */
    +public interface IBlobWatcher {
    +    public void blobChanged(String key);
    +}
    --- End diff --
    
    @d2r removed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45124638
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -24,9 +24,14 @@
                [org.apache.commons.io FileUtils]
                [java.io File])
       (:use [backtype.storm config util log timer local-state])
    +  (:import [backtype.storm.generated AuthorizationException KeyNotFoundException WorkerResources])
    +  (:import [java.util.concurrent Executors])
    --- End diff --
    
    Unused?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by unsleepy22 <gi...@git.apache.org>.
Github user unsleepy22 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46018606
  
    --- Diff: storm-core/src/clj/backtype/storm/cluster.clj ---
    @@ -90,9 +86,14 @@
       (update-storm! [this storm-id new-elems])
       (remove-storm-base! [this storm-id])
       (set-assignment! [this storm-id info])
    -  ;adds nimbusinfo under /stormroot/code-distributor/storm-id
    -  (setup-code-distributor! [this storm-id info])
    +  ;; Sets up information related to key consisting of nimbus
    --- End diff --
    
    Sets --> sets? keep consistent with the comment style above, same for all below.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45760956
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/KeyVersion.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * 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.zookeeper.CreateMode;
    +import org.apache.zookeeper.ZooDefs;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.TreeSet;
    +import java.util.Map;
    +import java.util.List;
    +
    +/**
    + * Class hands over the version of the key to be stored within the zookeeper
    + */
    +public class KeyVersion {
    +  private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
    +  private final String BLOBSTORE_SUBTREE="/blobstore";
    +  private final String BLOBSTORE_KEY_COUNTER_SUBTREE="/blobstorekeycounter";
    +  private String key;
    +  private NimbusInfo nimbusInfo;
    +
    +  public KeyVersion(String key, NimbusInfo nimbusInfo) {
    +    this.key = key;
    +    this.nimbusInfo = nimbusInfo;
    +  }
    +
    +  public int getKeyVersion(Map conf) {
    +    TreeSet<Integer> versions = new TreeSet<Integer>();
    +    CuratorFramework zkClient = Utils.createZKClient(conf);
    +    try {
    +      // Key has not been created yet and it is the first time it is being created
    +      if(zkClient.checkExists().forPath(BLOBSTORE_SUBTREE + "/" + key) == null) {
    +        zkClient.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT)
    +                .withACL(ZooDefs.Ids.OPEN_ACL_UNSAFE).forPath(BLOBSTORE_KEY_COUNTER_SUBTREE + "/" + key + "/" + 1);
    +        return 1;
    +      }
    +
    +      // When all nimbodes go down and one or few of them come up
    +      // Unfortunately there might not be an exact way to know which one contains the most updated blob
    +      // if all go down which is unlikely. Hence there might be a need to update the blob if all go down
    +      List<String> stateInfoList = zkClient.getChildren().forPath(BLOBSTORE_SUBTREE + "/" + key);
    +      LOG.debug("stateInfoList {} stateInfoList {}", stateInfoList.size(), stateInfoList);
    +      if(stateInfoList.isEmpty()) {
    +        return getKeyVersionCounterValue(zkClient, key);
    +      }
    +
    +      LOG.debug("stateInfoSize {}", stateInfoList.size());
    +      // In all other cases check for the latest version on the nimbus and assign the version
    +      // check if all are have same version, if not assign the highest version
    +      for (String stateInfo:stateInfoList) {
    +        versions.add(Integer.parseInt(Utils.normalizeVersionInfo(stateInfo)[1]));
    +      }
    +
    +      int currentCounter = getKeyVersionCounterValue(zkClient, key);
    +      // This condition returns version when a nimbus crashes and comes up
    +      if (!checkIfStateContainsCurrentNimbusHost(stateInfoList, nimbusInfo) && !nimbusInfo.isLeader()) {
    +        if (versions.last() < currentCounter) {
    +          return currentCounter;
    +        } else {
    +          return currentCounter - 1;
    --- End diff --
    
    @d2r addressed Actually it does have to be currentCounter - 1. It an return 0, the idea is when nimbus crashes and comes up it has to download the latest version of the blob


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r44995641
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -326,16 +331,29 @@
                                              (log-error t "Error when processing event")
                                              (exit-process! 20 "Error when processing an event")
                                              ))
    +   :blob-update-timer (mk-timer :kill-fn (fn [t]
    +                                           (log-error t "Error when processing blob-update")
    +                                           (exit-process! 20 "Error when processing a blob-update")
    +                                           ))
    +   :localizer (Utils/createLocalizer conf (supervisor-local-dir conf))
        :assignment-versions (atom {})
        :sync-retry (atom 0)
    -   :code-distributor (mk-code-distributor conf)
        :download-lock (Object.)
        :stormid->profiler-actions (atom {})
        })
     
    +(defn required-topo-files-exist?
    +  [conf storm-id]
    +  (let [stormroot (supervisor-stormdist-root conf storm-id)
    +        stormjarpath (supervisor-stormjar-path stormroot)
    +        stormcodepath (supervisor-stormcode-path stormroot)
    +        stormconfpath (supervisor-stormconf-path stormroot)]
    +    (and (every? exists-file? [stormroot stormconfpath stormcodepath])
    +      (or (local-mode? conf)
    +        (exists-file? stormjarpath)))))
    --- End diff --
    
    Indentation on 352 and 353 is wrong. Should be inline with previous args.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45561354
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStoreAclHandler.java ---
    @@ -0,0 +1,387 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.AccessControl;
    +import backtype.storm.generated.AccessControlType;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.security.auth.AuthUtils;
    +import backtype.storm.security.auth.IPrincipalToLocal;
    +import backtype.storm.security.auth.NimbusPrincipal;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.security.Principal;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Provides common handling of acls for Blobstores.
    + * Also contains some static utility functions related to Blobstores.
    + */
    +public class BlobStoreAclHandler {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStoreAclHandler.class);
    +  private final IPrincipalToLocal _ptol;
    +
    +  public static final int READ = 0x01;
    +  public static final int WRITE = 0x02;
    +  public static final int ADMIN = 0x04;
    +  public static final List<AccessControl> WORLD_EVERYTHING =
    +      Arrays.asList(new AccessControl(AccessControlType.OTHER, READ | WRITE | ADMIN));
    +  public static final List<AccessControl> DEFAULT = new ArrayList<AccessControl>();
    +  private Set<String> _supervisors;
    +  private Set<String> _admins;
    +
    +  public BlobStoreAclHandler(Map conf) {
    +    _ptol = AuthUtils.GetPrincipalToLocalPlugin(conf);
    +    _supervisors = new HashSet<String>();
    +    _admins = new HashSet<String>();
    +    if (conf.containsKey(Config.NIMBUS_SUPERVISOR_USERS)) {
    +      _supervisors.addAll((List<String>)conf.get(Config.NIMBUS_SUPERVISOR_USERS));
    +    }
    +    if (conf.containsKey(Config.NIMBUS_ADMINS)) {
    +      _admins.addAll((List<String>)conf.get(Config.NIMBUS_ADMINS));
    +    }
    +  }
    +
    +  private static AccessControlType parseACLType(String type) {
    +    if ("other".equalsIgnoreCase(type) || "o".equalsIgnoreCase(type)) {
    +      return AccessControlType.OTHER;
    +    } else if ("user".equalsIgnoreCase(type) || "u".equalsIgnoreCase(type)) {
    +      return AccessControlType.USER;
    +    }
    +    throw new IllegalArgumentException(type+" is not a valid access control type");
    +  }
    +
    +  private static int parseAccess(String access) {
    +    int ret = 0;
    +    for (char c: access.toCharArray()) {
    +      if ('r' == c) {
    +        ret = ret | READ;
    +      } else if ('w' == c) {
    +        ret = ret | WRITE;
    +      } else if ('a' == c) {
    +        ret = ret | ADMIN;
    +      } else if ('-' == c) {
    +        //ignored
    +      } else {
    +        throw new IllegalArgumentException("");
    +      }
    +    }
    +    return ret;
    +  }
    +
    +  public static AccessControl parseAccessControl(String str) {
    +    String[] parts = str.split(":");
    +    String type = "other";
    +    String name = "";
    +    String access = "-";
    +    if (parts.length > 3) {
    +      throw new IllegalArgumentException("Don't know how to parse "+str+" into an ACL value");
    +    } else if (parts.length == 1) {
    +      type = "other";
    +      name = "";
    +      access = parts[0];
    +    } else if (parts.length == 2) {
    +      type = "user";
    +      name = parts[0];
    +      access = parts[1];
    +    } else if (parts.length == 3) {
    +      type = parts[0];
    +      name = parts[1];
    +      access = parts[2];
    +    }
    +    AccessControl ret = new AccessControl();
    +    ret.set_type(parseACLType(type));
    +    ret.set_name(name);
    +    ret.set_access(parseAccess(access));
    +    return ret;
    +  }
    +
    +  private static String accessToString(int access) {
    +    StringBuffer ret = new StringBuffer();
    +    ret.append(((access & READ) > 0) ? "r" : "-");
    +    ret.append(((access & WRITE) > 0) ? "w" : "-");
    +    ret.append(((access & ADMIN) > 0) ? "a" : "-");
    +    return ret.toString();
    +  }
    +
    +  public static String accessControlToString(AccessControl ac) {
    +    StringBuffer ret = new StringBuffer();
    +    switch(ac.get_type()) {
    +      case OTHER:
    +        ret.append("o");
    +        break;
    +      case USER:
    +        ret.append("u");
    +        break;
    +      default:
    +        throw new IllegalArgumentException("Ahh don't know what a type of "+ac.get_type()+" means ");
    +    }
    +    ret.append(":");
    +    if (ac.is_set_name()) {
    +      ret.append(ac.get_name());
    +    }
    +    ret.append(":");
    +    ret.append(accessToString(ac.get_access()));
    +    return ret.toString();
    +  }
    +
    +  public static void validateSettableACLs(String key, List<AccessControl> acls) throws AuthorizationException {
    +    Set<String> aclUsers = new HashSet<>();
    +    List<String> duplicateUsers = new ArrayList<>();
    +    for (AccessControl acl : acls) {
    +      String aclUser = acl.get_name();
    +      if (aclUser != null && !aclUser.isEmpty() && !aclUsers.add(aclUser)) {
    +        LOG.error("'{}' user can't appear more than once in the ACLs", aclUser);
    +        duplicateUsers.add(aclUser);
    +      }
    +    }
    +    if (duplicateUsers.size() > 0) {
    +      String errorMessage  = "user " + Arrays.toString(duplicateUsers.toArray())
    +          + " can't appear more than once in the ACLs for key [" + key +"].";
    +      throw new AuthorizationException(errorMessage);
    +    }
    +  }
    +
    +  private Set<String> constructUserFromPrincipals(Subject who) {
    +    Set<String> user = new HashSet<String>();
    +    if (who == null) {
    +      LOG.debug("in validate acl who is null");
    +    } else {
    +      LOG.debug("in validate acl: " + who);
    +    }
    +    if (who != null) {
    +      for (Principal p : who.getPrincipals()) {
    +        user.add(_ptol.toLocal(p));
    +      }
    +    }
    +    return user;
    +  }
    +
    +  private boolean isSupervisorOrAdmin(Set<String> user, int mask) {
    +    boolean isSupervisor = false;
    +    boolean isAdmin = false;
    +    for(String u : user) {
    +      if (_supervisors.contains(u)) {
    +        isSupervisor = true;
    +        break;
    +      }
    +      if (_admins.contains(u)) {
    +        isAdmin = true;
    +        break;
    +      }
    +    }
    +    if (mask > 0 && !isAdmin) {
    +      isSupervisor = (isSupervisor && (mask == 1));
    +    }
    +    return isSupervisor || isAdmin;
    +  }
    +
    +  private boolean isNimbus(Subject who) {
    +    Set<Principal> principals = null;
    +    boolean isNimbusInstance = false;
    +    if(who != null) {
    +      principals = who.getPrincipals();
    +      for (Principal principal : principals) {
    +        if (principal instanceof NimbusPrincipal) {
    +          isNimbusInstance = true;
    +        }
    +      }
    +    }
    +    return isNimbusInstance;
    +  }
    +
    +  /**
    +   * The user should be able to see the metadata if and only if they have any of READ, WRITE, or ADMIN
    +   */
    +  public void validateUserCanReadMeta(List<AccessControl> acl, Subject who, String key)
    +      throws AuthorizationException {
    +    Set<String> user = constructUserFromPrincipals(who);
    +    if (isNimbus(who)) {
    +      return;
    +    }
    +    if (isSupervisorOrAdmin(user, -1)) {
    +      return;
    +    }
    +    for (AccessControl ac : acl) {
    +      int allowed = getAllowed(ac, user);
    +      LOG.debug(" user: {} allowed: {} key: {}", user, allowed, key);
    +      if ((allowed & (READ | WRITE | ADMIN)) > 0) {
    +        return;
    +      }
    +    }
    +    throw new AuthorizationException(
    +            user + " does not have access to " + key);
    +  }
    +
    +  public void validateAnyACL(List<AccessControl> acl, int validPermissions, Subject who, String key) throws AuthorizationException {
    +    Set<String> user = constructUserFromPrincipals(who);
    +    LOG.debug("user {}", user);
    +    if (isNimbus(who)) {
    +      return;
    +    }
    +    if(isSupervisorOrAdmin(user, validPermissions)) {
    +      return;
    +    }
    +    for (AccessControl ac : acl) {
    +      int allowed = getAllowed(ac, user);
    +      LOG.debug(" user: {} allowed: {} key: {}", user, allowed, key);
    +      if ((allowed & validPermissions) > 0) {
    +        return;
    +      }
    +    }
    +    throw new AuthorizationException(
    +            user + " does not have access to " + key);
    +  }
    +
    +  //Here all acls must match
    +  public void validateACL(List<AccessControl> acl, int mask, Subject who, String key)
    +      throws AuthorizationException {
    +    Set<String> user = constructUserFromPrincipals(who);
    +    LOG.debug("user {}", user);
    +    if (isNimbus(who)) {
    +      return;
    +    }
    +    if(isSupervisorOrAdmin(user, mask)) {
    +      return;
    +    }
    +    for (AccessControl ac : acl) {
    +      int allowed = getAllowed(ac, user);
    +      mask = ~allowed & mask;
    +      LOG.debug(" user: {} allowed: {} disallowed: {} key: {}", user, allowed, mask, key);
    +    }
    +    if (mask == 0) {
    +      return;
    +    }
    +    throw new AuthorizationException(
    +            user + " does not have " + namedPerms(mask) + " access to " + key);
    +  }
    --- End diff --
    
    @d2r Made the changes, hasPermissions and hasAnyPermissions for validateACL and validateAnyACL respectively and refactored the code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by unsleepy22 <gi...@git.apache.org>.
Github user unsleepy22 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46022785
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStoreUtils.java ---
    @@ -0,0 +1,256 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.security.auth.NimbusPrincipal;
    +import backtype.storm.utils.NimbusClient;
    +import backtype.storm.utils.Utils;
    +import backtype.storm.utils.ZookeeperAuthInfo;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.thrift.transport.TTransportException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +public class BlobStoreUtils {
    +    private static final String BLOBSTORE_SUBTREE="/blobstore";
    +    private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
    +
    +    public static CuratorFramework createZKClient(Map conf) {
    +        List<String> zkServers = (List<String>) conf.get(Config.STORM_ZOOKEEPER_SERVERS);
    +        Object port = conf.get(Config.STORM_ZOOKEEPER_PORT);
    +        ZookeeperAuthInfo zkAuthInfo = new ZookeeperAuthInfo(conf);
    +        CuratorFramework zkClient = Utils.newCurator(conf, zkServers, port, (String) conf.get(Config.STORM_ZOOKEEPER_ROOT), zkAuthInfo);
    +        zkClient.start();
    +        return zkClient;
    +    }
    +
    +    public static Subject getNimbusSubject() {
    +        Subject subject = new Subject();
    +        subject.getPrincipals().add(new NimbusPrincipal());
    +        return subject;
    +    }
    +
    +    // Normalize state
    +    public static BlobKeySequenceInfo normalizeNimbusHostPortSequenceNumberInfo(String nimbusSeqNumberInfo) {
    +        BlobKeySequenceInfo keySequenceInfo = new BlobKeySequenceInfo();
    +        int lastIndex = nimbusSeqNumberInfo.lastIndexOf("-");
    +        keySequenceInfo.setNimbusHostPort(nimbusSeqNumberInfo.substring(0, lastIndex));
    +        keySequenceInfo.setSequenceNumber(nimbusSeqNumberInfo.substring(lastIndex + 1));
    +        return keySequenceInfo;
    +    }
    +
    +    // Check for latest sequence number of a key inside zookeeper and return nimbodes containing the latest sequence number
    +    public static Set<NimbusInfo> getNimbodesWithLatestSequenceNumberOfBlob(CuratorFramework zkClient, String key) throws Exception {
    +        List<String> stateInfoList = zkClient.getChildren().forPath("/blobstore/" + key);
    +        Set<NimbusInfo> nimbusInfoSet = new HashSet<NimbusInfo>();
    +        int latestSeqNumber = getLatestSequenceNumber(stateInfoList);
    +        LOG.debug("getNimbodesWithLatestSequenceNumberOfBlob stateInfo {} version {}", stateInfoList, latestSeqNumber);
    +        // Get the nimbodes with the latest version
    +        for(String state : stateInfoList) {
    +            BlobKeySequenceInfo sequenceInfo = normalizeNimbusHostPortSequenceNumberInfo(state);
    +            if (latestSeqNumber == Integer.parseInt(sequenceInfo.getSequenceNumber())) {
    +                nimbusInfoSet.add(NimbusInfo.parse(sequenceInfo.getNimbusHostPort()));
    +            }
    +        }
    +        LOG.debug("nimbusInfoList {}", nimbusInfoSet);
    +        return nimbusInfoSet;
    +    }
    +
    +    // Get sequence number details from latest sequence number of the blob
    +    public static int getLatestSequenceNumber(List<String> stateInfoList) {
    +        int seqNumber = 0;
    +        // Get latest sequence number of the blob present in the zookeeper --> possible to refactor this piece of code
    +        for (String state : stateInfoList) {
    +            BlobKeySequenceInfo sequenceInfo = normalizeNimbusHostPortSequenceNumberInfo(state);
    +            if (seqNumber < Integer.parseInt(sequenceInfo.getSequenceNumber())) {
    +                seqNumber = Integer.parseInt(sequenceInfo.getSequenceNumber());
    +                LOG.debug("Sequence Info {}", seqNumber);
    +            }
    +        }
    +        LOG.debug("Latest Sequence Number {}", seqNumber);
    +        return seqNumber;
    +    }
    +
    +    // Download missing blobs from potential nimbodes
    +    public static boolean downloadMissingBlob(Map conf, BlobStore blobStore, String key, Set<NimbusInfo> nimbusInfos)
    +            throws TTransportException {
    +        NimbusClient client = null;
    +        ReadableBlobMeta rbm = null;
    +        ClientBlobStore remoteBlobStore = null;
    +        InputStreamWithMeta in = null;
    +        Boolean isSuccess = false;
    +        LOG.debug("Download blob NimbusInfos {}", nimbusInfos);
    +        for (NimbusInfo nimbusInfo : nimbusInfos) {
    +            if(isSuccess) {
    +                break;
    +            }
    +            try {
    +                client = new NimbusClient(conf, nimbusInfo.getHost(), nimbusInfo.getPort(), null);
    +                rbm = client.getClient().getBlobMeta(key);
    +                remoteBlobStore = new NimbusBlobStore();
    +                remoteBlobStore.setClient(conf, client);
    +                in = remoteBlobStore.getBlob(key);
    +                blobStore.createBlob(key, in, rbm.get_settable(), getNimbusSubject());
    +                // if key already exists while creating the blob else update it
    +                Iterator<String> keyIterator = blobStore.listKeys();
    +                while (keyIterator.hasNext()) {
    +                    if (keyIterator.next().equals(key)) {
    +                        LOG.debug("Success creating key, {}", key);
    +                        isSuccess = true;
    +                        break;
    +                    }
    +                }
    +            } catch (IOException | AuthorizationException exception) {
    +                throw new RuntimeException(exception);
    +            } catch (KeyAlreadyExistsException kae) {
    +                LOG.info("KeyAlreadyExistsException Key: {} {}", key, kae);
    +            } catch (KeyNotFoundException knf) {
    +                // Catching and logging KeyNotFoundException because, if
    +                // there is a subsequent update and delete, the non-leader
    +                // nimbodes might throw an exception.
    +                LOG.info("KeyNotFoundException Key: {} {}", key, knf);
    +            } catch (Exception exp) {
    +                // Logging an exception while client is connecting
    +                LOG.error("Exception {}", exp);
    +            }
    +        }
    +
    +        if (!isSuccess) {
    +            LOG.error("Could not download blob with key" + key);
    +            return false;
    +        }
    +        return isSuccess;
    +    }
    +
    +    // Download updated blobs from potential nimbodes
    +    public static boolean downloadUpdatedBlob(Map conf, BlobStore blobStore, String key, Set<NimbusInfo> nimbusInfos)
    +            throws TTransportException {
    +        NimbusClient client = null;
    +        ClientBlobStore remoteBlobStore = null;
    +        InputStreamWithMeta in = null;
    +        AtomicOutputStream out = null;
    +        Boolean isSuccess = false;
    --- End diff --
    
    boolean


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45272835
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStore.java ---
    @@ -0,0 +1,446 @@
    +/**
    + * 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 java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Set;
    +import java.util.regex.Pattern;
    +
    +import javax.security.auth.Subject;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import backtype.storm.daemon.Shutdownable;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +
    +/**
    + * Provides a way to store blobs that can be downloaded.
    + * Blobs must be able to be uploaded and listed from Nimbus,
    + * and downloaded from the Supervisors. It is a key value based
    + * store. Key being a string and value being the blob data.
    + *
    + * ACL checking must take place against the provided subject.
    + * If the blob store does not support Security it must validate
    + * that all ACLs set are always WORLD, everything.
    + *
    + * The users can upload their blobs through the blob store command
    + * line utility. The command line utilty also allows us to update,
    + * delete.
    + *
    + * Modifying the replication factor only works for HdfsBlobStore
    + * as for the LocalFsBlobStore the replication is dependent on
    + * the number of Nimbodes available.
    + */
    +public abstract class BlobStore implements Shutdownable {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStore.class);
    +  private static final Pattern KEY_PATTERN = Pattern.compile("^[\\w \\t\\.:_-]+$");
    +  protected static final String BASE_BLOBS_DIR_NAME = "blobs";
    +
    +  /**
    +   * Allows us to initialize the blob store
    +   * @param conf
    +   * @param baseDir
    +   */
    +  public abstract void prepare(Map conf, String baseDir, NimbusInfo nimbusInfo);
    +
    +  /**
    +   * Creates the blob.
    +   * @param key Key for the blob.
    +   * @param meta Metadata which contains the acls information
    +   * @param who Is the subject creating the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyAlreadyExistsException
    +   */
    +  public abstract AtomicOutputStream createBlob(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException;
    +
    +  /**
    +   * Updates the blob data.
    +   * @param key Key for the blob.
    +   * @param who Is the subject creating the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract AtomicOutputStream updateBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Gets the current version of metadata for a blob
    +   * to be viewed by the user or downloaded by the supervisor.
    +   * @param key Key for the blob.
    +   * @param who Is the subject creating the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract ReadableBlobMeta getBlobMeta(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Sets the metadata with renewed acls for the blob.
    +   * @param key Key for the blob.
    +   * @param meta Metadata which contains the updated
    +   * acls information.
    +   * @param who Is the subject creating the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract void setBlobMeta(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Deletes the blob data and metadata.
    +   * @param key Key for the blob.
    +   * @param who Is the subject creating the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract void deleteBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Gets the current version of metadata for a blob
    +   * to be viewed by the user or downloaded by the supervisor.
    +   * @param key Key for the blob.
    +   * @param who Is the subject creating the blob.
    +   * @return InputStreamWithMeta has the additional
    +   * file length and version information.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract InputStreamWithMeta getBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Returns an iterator with all the list of
    +   * keys currently available on the blob store.
    +   * @param who Is the subject creating the blob.
    +   * @return Iterator<String>
    +   */
    +  public abstract Iterator<String> listKeys(Subject who);
    --- End diff --
    
    What is `who` used for?  We aren't creating a blob.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43809808
  
    --- Diff: storm-core/src/clj/backtype/storm/zookeeper.clj ---
    @@ -186,6 +186,21 @@
           (.. zk (getChildren) (forPath (normalize-path path))))
         (catch Exception e (throw (wrap-in-runtime e)))))
     
    +;; Deletes the state inside the zookeeper for a key, for which the
    +;; contents of the key starts with nimbus host port information
    --- End diff --
    
    Let's turn these comments into a docstring for the function.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r44997502
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -732,25 +863,89 @@
      (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
       (worker-launcher-and-wait conf (storm-conf TOPOLOGY-SUBMITTER-USER) ["code-dir" dir] :log-prefix (str "setup conf for " dir))))
     
    +(defn setup-blob-permission [conf storm-conf path]
    +  (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
    +    (worker-launcher-and-wait conf (storm-conf TOPOLOGY-SUBMITTER-USER) ["blob" path] :log-prefix (str "setup blob permissions for " path))))
    +
    +(defn setup-storm-code-dir [conf storm-conf dir]
    +  (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
    +    (worker-launcher-and-wait conf (storm-conf TOPOLOGY-SUBMITTER-USER) ["code-dir" dir] :log-prefix (str "setup conf for " dir))))
    --- End diff --
    
    It might be nice to break these function calls up, but that's really nit-picky. Not a big deal at all.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45266348
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -927,31 +1133,32 @@
            first ))
     
     (defmethod download-storm-code
    -    :local [conf storm-id master-code-dir supervisor download-lock]
    -    (let [stormroot (supervisor-stormdist-root conf storm-id)]
    -      (locking download-lock
    -            (FileUtils/copyDirectory (File. master-code-dir) (File. stormroot))
    -            (let [classloader (.getContextClassLoader (Thread/currentThread))
    -                  resources-jar (resources-jar)
    -                  url (.getResource classloader RESOURCES-SUBDIR)
    -                  target-dir (str stormroot file-path-separator RESOURCES-SUBDIR)]
    -              (cond
    -               resources-jar
    -               (do
    -                 (log-message "Extracting resources from jar at " resources-jar " to " target-dir)
    -                 (extract-dir-from-jar resources-jar RESOURCES-SUBDIR stormroot))
    -               url
    -               (do
    -                 (log-message "Copying resources at " (URI. (str url)) " to " target-dir)
    -                 (if (= (.getProtocol url) "jar" )
    -                   (extract-dir-from-jar (.getFile (.getJarFileURL (.openConnection url))) RESOURCES-SUBDIR stormroot)
    -                   (FileUtils/copyDirectory (File. (.getPath (URI. (str url)))) (File. target-dir)))
    -                 )
    -               )
    -              )
    -            )))
    -
    -(defmethod mk-code-distributor :local [conf] nil)
    +  :local [conf storm-id master-code-dir localizer]
    +  (let [tmproot (str (supervisor-tmp-dir conf) file-path-separator (uuid))
    +        stormroot (supervisor-stormdist-root conf storm-id)
    +        blob-store (Utils/getNimbusBlobStore conf master-code-dir nil)]
    +    (try
    +      (FileUtils/forceMkdir (File. tmproot))
    +      (.readBlobTo blob-store (master-stormcode-key storm-id) (FileOutputStream. (supervisor-stormcode-path tmproot)) nil)
    +      (.readBlobTo blob-store (master-stormconf-key storm-id) (FileOutputStream. (supervisor-stormconf-path tmproot)) nil)
    +      (finally
    +        (.shutdown blob-store)))
    +    (FileUtils/moveDirectory (File. tmproot) (File. stormroot))
    +    (setup-storm-code-dir conf (read-supervisor-storm-conf conf storm-id) stormroot)
    +    (let [classloader (.getContextClassLoader (Thread/currentThread))
    +          resources-jar (resources-jar)
    +          url (.getResource classloader RESOURCES-SUBDIR)
    +          target-dir (str stormroot file-path-separator RESOURCES-SUBDIR)]
    +      (cond
    +        resources-jar
    --- End diff --
    
    > param localizer is not used
    
    The `defmulti` `download-storm-code` has two methods, one for `:local` and one for `:distributed`. Both methods take the same number of parameters (4), but only use 3 of them. So in both methods there is an unused parameter.
    
    We could change to using 3 parameters, and just call the method with the correct third parameter (which we should know at the time we call it).
    
    ```Clojure
    (defmethod download-storm-code :local
      [conf storm-id master-code-dir]
    ```
    
    and 
    
    ```Clojure
    (defmethod download-storm-code :distributed
      [conf storm-id localizer]
    ```
    
    The dispatch function is `config/cluster-mode` and it only looks at the first argument (the conf).



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45425150
  
    --- Diff: external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java ---
    @@ -0,0 +1,383 @@
    +/**
    + * 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 org.apache.storm.hdfs.blobstore;
    +
    +import backtype.storm.Config;
    +import backtype.storm.blobstore.AtomicOutputStream;
    +import backtype.storm.blobstore.AtomicOutputStream;
    +import backtype.storm.blobstore.BlobStore;
    +import backtype.storm.blobstore.BlobStoreAclHandler;
    +import backtype.storm.blobstore.BlobStoreFile;
    +import backtype.storm.blobstore.InputStreamWithMeta;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.utils.NimbusClient;
    --- End diff --
    
    Unused?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45896515
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStore.java ---
    @@ -0,0 +1,297 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.utils.Utils;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.thrift.TBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.ByteArrayOutputStream;
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.FileNotFoundException;
    +import java.io.InputStream;
    +
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;;
    +
    +import static backtype.storm.blobstore.BlobStoreAclHandler.ADMIN;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.READ;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.WRITE;
    +
    +/**
    + * Provides a local file system backed blob store implementation for Nimbus.
    + */
    +public class LocalFsBlobStore extends BlobStore {
    +  public static final Logger LOG = LoggerFactory.getLogger(LocalFsBlobStore.class);
    +  private static final String DATA_PREFIX = "data_";
    +  private static final String META_PREFIX = "meta_";
    +  protected BlobStoreAclHandler _aclHandler;
    +  private final String BLOBSTORE_SUBTREE = "/blobstore/";
    +  private NimbusInfo nimbusInfo;
    +  private FileBlobStoreImpl fbs;
    +  private Map conf;
    +
    +  @Override
    +  public void prepare(Map conf, String overrideBase, NimbusInfo nimbusInfo) {
    +    this.conf = conf;
    +    this.nimbusInfo = nimbusInfo;
    +     if (overrideBase == null) {
    +      overrideBase = (String)conf.get(Config.BLOBSTORE_DIR);
    +      if (overrideBase == null) {
    +        overrideBase = (String) conf.get(Config.STORM_LOCAL_DIR);
    +      }
    +    }
    +    File baseDir = new File(overrideBase, BASE_BLOBS_DIR_NAME);
    +    try {
    +      fbs = new FileBlobStoreImpl(baseDir, conf);
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +    _aclHandler = new BlobStoreAclHandler(conf);
    +  }
    --- End diff --
    
    @d2r addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45494751
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/KeyVersion.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * 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.zookeeper.CreateMode;
    +import org.apache.zookeeper.ZooDefs;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.TreeSet;
    +import java.util.Map;
    +import java.util.List;
    +
    +/**
    + * Class hands over the version of the key to be stored within the zookeeper
    + */
    +public class KeyVersion {
    +  private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
    +  private final String BLOBSTORE_SUBTREE="/blobstore";
    +  private final String BLOBSTORE_KEY_COUNTER_SUBTREE="/blobstorekeycounter";
    +  private String key;
    +  private NimbusInfo nimbusInfo;
    +
    +  public KeyVersion(String key, NimbusInfo nimbusInfo) {
    +    this.key = key;
    +    this.nimbusInfo = nimbusInfo;
    +  }
    +
    +  public int getKeyVersion(Map conf) {
    +    TreeSet<Integer> versions = new TreeSet<Integer>();
    +    CuratorFramework zkClient = Utils.createZKClient(conf);
    +    try {
    +      // Key has not been created yet and it is the first time it is being created
    +      if(zkClient.checkExists().forPath(BLOBSTORE_SUBTREE + "/" + key) == null) {
    +        zkClient.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT)
    +                .withACL(ZooDefs.Ids.OPEN_ACL_UNSAFE).forPath(BLOBSTORE_KEY_COUNTER_SUBTREE + "/" + key + "/" + 1);
    +        return 1;
    +      }
    +
    +      // When all nimbodes go down and one or few of them come up
    +      // Unfortunately there might not be an exact way to know which one contains the most updated blob
    +      // if all go down which is unlikely. Hence there might be a need to update the blob if all go down
    +      List<String> stateInfoList = zkClient.getChildren().forPath(BLOBSTORE_SUBTREE + "/" + key);
    +      LOG.debug("stateInfoList {} stateInfoList {}", stateInfoList.size(), stateInfoList);
    +      if(stateInfoList.isEmpty()) {
    +        return getKeyVersionCounterValue(zkClient, key);
    +      }
    +
    +      LOG.debug("stateInfoSize {}", stateInfoList.size());
    +      // In all other cases check for the latest version on the nimbus and assign the version
    +      // check if all are have same version, if not assign the highest version
    +      for (String stateInfo:stateInfoList) {
    +        versions.add(Integer.parseInt(Utils.normalizeVersionInfo(stateInfo)[1]));
    +      }
    +
    +      int currentCounter = getKeyVersionCounterValue(zkClient, key);
    +      // This condition returns version when a nimbus crashes and comes up
    +      if (!checkIfStateContainsCurrentNimbusHost(stateInfoList, nimbusInfo) && !nimbusInfo.isLeader()) {
    +        if (versions.last() < currentCounter) {
    +          return currentCounter;
    +        } else {
    +          return currentCounter - 1;
    +        }
    +      }
    +      // Condition checks for an update scenario
    +      if (stateInfoList.size() >= 1 && versions.size() == 1) {
    +        if (versions.first() < getKeyVersionCounterValue(zkClient, key)) {
    +          incrementCounter(zkClient, key, currentCounter);
    +          return currentCounter + 1;
    +        } else {
    +          incrementCounter(zkClient, key, currentCounter);
    +          return versions.first() + 1;
    --- End diff --
    
    OK, I think the documentation would help clarify it.
    * What we are calling "version" here is not the same version as in the public API exposed via thrift.  We should call it something like "sequence number" to avoid confusion
    * The "counter" values here do correspond to the sequence numbers; they are not independent.  We should rename the "counter" to be something like "max-sequence-number," because that is what it represents.
    * For the max-sequence-number znode, currently we are creating a znode with no data whose name is the value of the sequence number.  When we need to change it, we delete the old znode and create a new one.  I don't think this is atomic.  Instead we can create a permanent single znode named something like max-sequence-number and set the number we want in the data of that znode.  I think that should be atomic.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45101369
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -529,10 +628,11 @@
     
     (defn- compute-executors [nimbus storm-id]
       (let [conf (:conf nimbus)
    + 	      blob-store (:blob-store nimbus)
    --- End diff --
    
    check white-space


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45123190
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -1767,6 +1900,169 @@
                                       topology-id
                                       (doto (GetInfoOptions.) (.set_num_err_choice NumErrorsChoice/ALL))))
     
    +      ;;Blobstore implementation code
    +      (^String beginCreateBlob [this
    +                                ^String blob-key
    +                                ^SettableBlobMeta blob-meta]
    +        (let [session-id (uuid)]
    +          (.put (:blob-uploaders nimbus)
    +            session-id
    +            (->> (ReqContext/context)
    +              (.subject)
    +              (.createBlob (:blob-store nimbus) blob-key blob-meta)))
    --- End diff --
    
    Here and elsewhere, use helper function `get-subject` instead of code like `(->> (ReqContext/context) (.subject))`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the pull request:

    https://github.com/apache/storm/pull/845#issuecomment-160761949
  
    Things look good to me.  I am +1 on this going in.  I had one comment about getting some documentation for the ClientBlobStore.java interface.  But that can go in later if we need to.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43788727
  
    --- Diff: bin/storm.py ---
    @@ -263,6 +263,34 @@ def upload_credentials(*args):
             jvmtype="-client",
             extrajars=[USER_CONF_DIR, STORM_BIN_DIR])
     
    +def blobstore(*args):
    +    """Syntax: [storm blobstore [cmd]]
    +
    +    list [KEY...] - lists blobs currently in the blob store
    +    cat [-f FILE] KEY - read a blob and then either write it to a file, or STDOUT (requires read access).
    +    create [-f FILE] [-a ACL ...] [--repl-fctr NUMBER] KEY - create a new blob. Contents comes from a FILE
    +         or STDIN. ACL is in the form [uo]:[username]:[r-][w-][a-] can be comma
    +         separated list.
    +         --repl-fctr refers to replication factor for the blob. Here NUMBER > 0.
    +         for example the following would create 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.
    +         storm blobstore create mytopo:data.tgz -f data.tgz -a u:alice:rwa,u:bob:rw,o::r
    +    update [-f FILE] KEY - update the contents of a blob.  Contents comes from
    +         a FILE or STDIN (requires write access).
    +    delete KEY - delete an entry from the blob store (requires write access).
    +    set-acl [-s ACL] KEY - ACL is in the form [uo]:[username]:[r-][w-][a-] can be comma
    +         separated list (requires admin access).
    +    replication --read KEY - Used to read the replication factor of the blob.
    +    replication --update --repl-fctr NUMBER KEY - NUMBER > 0. It is used to update the
    --- End diff --
    
    Same here, rename to `--replication-factor`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45135495
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -454,10 +455,82 @@
           (shutdown-worker supervisor id))
         ))
     
    +(defn get-blob-localname
    +  "Given the blob information either gets the localname field if it exists,
    +  else routines the default value passed in."
    +  [blob-info defaultValue]
    +  (if-let [val (if blob-info (get blob-info "localname") nil)] val defaultValue))
    +
    +(defn should-uncompress-blob?
    +  "Given the blob information returns the value of the uncompress field, handling it either being
    +  a string or a boolean value, or ifs its not specified then returns false"
    +  [blob-info]
    +  (boolean (and blob-info
    +             (if-let [val (get blob-info "uncompress")]
    +               (.booleanValue (Boolean. val))))))
    +
    +(defn remove-blob-references
    +  "Remove a reference to a blob when its no longer needed."
    +  [localizer storm-id conf]
    +  (let [storm-conf (read-supervisor-storm-conf conf storm-id)
    +        blobstore-map (storm-conf TOPOLOGY-BLOBSTORE-MAP)
    +        user (storm-conf TOPOLOGY-SUBMITTER-USER)
    +        topo-name (storm-conf TOPOLOGY-NAME)]
    +    (if blobstore-map
    +      (doseq [[k, v] blobstore-map]
    +        (.removeBlobReference localizer
    +          k
    +          user
    +          topo-name
    +          (should-uncompress-blob? v))))))
    +
    +(defn blobstore-map-to-localresources
    +  "Returns a list of LocalResources based on the blobstore-map passed in."
    +  [blobstore-map]
    +  (if blobstore-map
    +    (for [[k, v] blobstore-map] (LocalResource. k (should-uncompress-blob? v)))
    +    ()))
    +
    +(defn add-blob-references
    +  "For each of the downloaded topologies, adds references to the blobs that the topologies are
    +  using. This is used to reconstruct the cache on restart."
    +  [localizer storm-id conf]
    +  (let [storm-conf (read-supervisor-storm-conf conf storm-id)
    +        blobstore-map (storm-conf TOPOLOGY-BLOBSTORE-MAP)
    +        user (storm-conf TOPOLOGY-SUBMITTER-USER)
    +        topo-name (storm-conf TOPOLOGY-NAME)
    +        localresources (blobstore-map-to-localresources blobstore-map)]
    +    (if blobstore-map
    +      (.addReferences localizer localresources user topo-name))))
    +
    +(defn rm-topo-files
    +  [conf storm-id localizer rm-blob-refs?]
    +  (try
    +    (if (= true rm-blob-refs?)
    +      (remove-blob-references localizer storm-id conf))
    --- End diff --
    
    Don't need `(= true`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43809952
  
    --- Diff: storm-core/src/clj/backtype/storm/zookeeper.clj ---
    @@ -186,6 +186,21 @@
           (.. zk (getChildren) (forPath (normalize-path path))))
         (catch Exception e (throw (wrap-in-runtime e)))))
     
    +;; Deletes the state inside the zookeeper for a key, for which the
    +;; contents of the key starts with nimbus host port information
    +(defn delete-node-blobstore
    +  [^CuratorFramework zk ^String parent-path ^String host-port-info]
    +  (let [parent-path (normalize-path parent-path)
    +        child-path-list (if (exists-node? zk parent-path false)
    +                          (into [] (get-children zk parent-path false))
    +                          [])]
    +    (doseq [child child-path-list]
    +      (if (.startsWith child host-port-info)
    +        (do
    +          (log-debug "delete-node " "child" child)
    +          (delete-node zk (str parent-path "/" child)))
    +        ))))
    --- End diff --
    
    Can use `when` instead of `if ... do`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45136151
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -454,10 +455,82 @@
           (shutdown-worker supervisor id))
         ))
     
    +(defn get-blob-localname
    +  "Given the blob information either gets the localname field if it exists,
    +  else routines the default value passed in."
    +  [blob-info defaultValue]
    +  (if-let [val (if blob-info (get blob-info "localname") nil)] val defaultValue))
    +
    +(defn should-uncompress-blob?
    +  "Given the blob information returns the value of the uncompress field, handling it either being
    +  a string or a boolean value, or ifs its not specified then returns false"
    +  [blob-info]
    +  (boolean (and blob-info
    +             (if-let [val (get blob-info "uncompress")]
    +               (.booleanValue (Boolean. val))))))
    +
    +(defn remove-blob-references
    +  "Remove a reference to a blob when its no longer needed."
    +  [localizer storm-id conf]
    +  (let [storm-conf (read-supervisor-storm-conf conf storm-id)
    +        blobstore-map (storm-conf TOPOLOGY-BLOBSTORE-MAP)
    +        user (storm-conf TOPOLOGY-SUBMITTER-USER)
    +        topo-name (storm-conf TOPOLOGY-NAME)]
    +    (if blobstore-map
    +      (doseq [[k, v] blobstore-map]
    +        (.removeBlobReference localizer
    +          k
    +          user
    +          topo-name
    +          (should-uncompress-blob? v))))))
    +
    +(defn blobstore-map-to-localresources
    +  "Returns a list of LocalResources based on the blobstore-map passed in."
    +  [blobstore-map]
    +  (if blobstore-map
    +    (for [[k, v] blobstore-map] (LocalResource. k (should-uncompress-blob? v)))
    +    ()))
    +
    +(defn add-blob-references
    +  "For each of the downloaded topologies, adds references to the blobs that the topologies are
    +  using. This is used to reconstruct the cache on restart."
    +  [localizer storm-id conf]
    +  (let [storm-conf (read-supervisor-storm-conf conf storm-id)
    +        blobstore-map (storm-conf TOPOLOGY-BLOBSTORE-MAP)
    +        user (storm-conf TOPOLOGY-SUBMITTER-USER)
    +        topo-name (storm-conf TOPOLOGY-NAME)
    +        localresources (blobstore-map-to-localresources blobstore-map)]
    +    (if blobstore-map
    +      (.addReferences localizer localresources user topo-name))))
    +
    +(defn rm-topo-files
    +  [conf storm-id localizer rm-blob-refs?]
    +  (try
    +    (if (= true rm-blob-refs?)
    +      (remove-blob-references localizer storm-id conf))
    +    (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
    +      (rmr-as-user conf storm-id (supervisor-stormdist-root conf storm-id))
    +      (rmr (supervisor-stormdist-root conf storm-id)))
    +    (catch Exception e
    +      (log-message e (str "Exception removing: " storm-id)))))
    +
    +(defn verify-downloaded-files [conf localizer assigned-storm-ids all-downloaded-storm-ids]
    +  "Method written to check for the files exists to avoid supervisor crashing
    +   Also makes sure there is no necessity for locking"
    --- End diff --
    
    Remove "Method written to"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43959604
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStoreAclHandler.java ---
    @@ -0,0 +1,382 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.AccessControl;
    +import backtype.storm.generated.AccessControlType;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.security.auth.AuthUtils;
    +import backtype.storm.security.auth.IPrincipalToLocal;
    +import backtype.storm.security.auth.NimbusPrincipal;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.security.Principal;
    +import java.util.*;
    +
    +/**
    + * Provides common handling of acls for Blobstores.
    + * Also contains some static utility functions related to Blobstores.
    + */
    +public class BlobStoreAclHandler {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStoreAclHandler.class);
    +  private final IPrincipalToLocal _ptol;
    +
    +  public static final int READ = 0x01;
    +  public static final int WRITE = 0x02;
    +  public static final int ADMIN = 0x04;
    +  public static final List<AccessControl> WORLD_EVERYTHING =
    +      Arrays.asList(new AccessControl(AccessControlType.OTHER, READ | WRITE | ADMIN));
    +  public static final List<AccessControl> DEFAULT = new ArrayList<AccessControl>();
    +  private Set<String> _supervisors;
    +  private Set<String> _admins;
    +
    +  public BlobStoreAclHandler(Map conf) {
    +    _ptol = AuthUtils.GetPrincipalToLocalPlugin(conf);
    +    _supervisors = new HashSet<String>();
    +    _admins = new HashSet<String>();
    +    if (conf.containsKey(Config.NIMBUS_SUPERVISOR_USERS)) {
    +      _supervisors.addAll((List<String>)conf.get(Config.NIMBUS_SUPERVISOR_USERS));
    +    }
    +    if (conf.containsKey(Config.NIMBUS_ADMINS)) {
    +      _admins.addAll((List<String>)conf.get(Config.NIMBUS_ADMINS));
    +    }
    +  }
    +
    +  private static AccessControlType parseACLType(String type) {
    +    if ("other".equalsIgnoreCase(type) || "o".equalsIgnoreCase(type)) {
    +      return AccessControlType.OTHER;
    +    } else if ("user".equalsIgnoreCase(type) || "u".equalsIgnoreCase(type)) {
    +      return AccessControlType.USER;
    +    }
    +    throw new IllegalArgumentException(type+" is not a valid access control type");
    +  }
    +
    +  private static int parseAccess(String access) {
    +    int ret = 0;
    +    for (char c: access.toCharArray()) {
    +      if ('r' == c) {
    +        ret = ret | READ;
    +      } else if ('w' == c) {
    +        ret = ret | WRITE;
    +      } else if ('a' == c) {
    +        ret = ret | ADMIN;
    +      } else if ('-' == c) {
    +        //ignored
    +      } else {
    +        throw new IllegalArgumentException("");
    +      }
    +    }
    +    return ret;
    +  }
    +
    +  public static AccessControl parseAccessControl(String str) {
    +    String[] parts = str.split(":");
    +    String type = "other";
    +    String name = "";
    +    String access = "-";
    +    if (parts.length > 3) {
    +      throw new IllegalArgumentException("Don't know how to parse "+str+" into an ACL value");
    +    } else if (parts.length == 1) {
    +      type = "other";
    +      name = "";
    +      access = parts[0];
    +    } else if (parts.length == 2) {
    +      type = "user";
    +      name = parts[0];
    +      access = parts[1];
    +    } else if (parts.length == 3) {
    +      type = parts[0];
    +      name = parts[1];
    +      access = parts[2];
    +    }
    +    AccessControl ret = new AccessControl();
    +    ret.set_type(parseACLType(type));
    +    ret.set_name(name);
    +    ret.set_access(parseAccess(access));
    +    return ret;
    +  }
    --- End diff --
    
    Will document it in DISTCACHE.markdown


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by kishorvpatil <gi...@git.apache.org>.
Github user kishorvpatil commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43821679
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStore.java ---
    @@ -0,0 +1,278 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.*;
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.utils.Utils;
    +import backtype.storm.utils.ZookeeperAuthInfo;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.thrift.TBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.*;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +
    +import static backtype.storm.blobstore.BlobStoreAclHandler.*;
    --- End diff --
    
    explicit import


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43810145
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -984,25 +1094,30 @@
             ))))
     
     (defn cleanup-corrupt-topologies! [nimbus]
    -  (if (is-leader nimbus :throw-exception false)
    -    (let [storm-cluster-state (:storm-cluster-state nimbus)
    -          code-ids (set (code-ids (:conf nimbus)))
    -          active-topologies (set (.active-storms storm-cluster-state))
    -          corrupt-topologies (set/difference active-topologies code-ids)]
    -      (doseq [corrupt corrupt-topologies]
    -        (log-message "Corrupt topology " corrupt " has state on zookeeper but doesn't have a local dir on Nimbus. Cleaning up...")
    -        (.remove-storm! storm-cluster-state corrupt)
    -        )))
    -  (log-message "not a leader, skipping cleanup-corrupt-topologies"))
    -
    -;;setsup code distributor entries for all current topologies for which code is available locally.
    -(defn setup-code-distributor [nimbus]
       (let [storm-cluster-state (:storm-cluster-state nimbus)
    -        locally-available-storm-ids (set (code-ids (:conf nimbus)))
    +        blob-store (:blob-store nimbus)
    +        code-ids (set (code-ids blob-store))
             active-topologies (set (.active-storms storm-cluster-state))
    -        locally-available-active-storm-ids (set/intersection locally-available-storm-ids active-topologies)]
    -    (doseq [storm-id locally-available-active-storm-ids]
    -      (.setup-code-distributor! storm-cluster-state storm-id (:nimbus-host-port-info nimbus)))))
    +        corrupt-topologies (set/difference active-topologies code-ids)]
    +    (doseq [corrupt corrupt-topologies]
    +      (log-message "Corrupt topology " corrupt " has state on zookeeper but doesn't have a local dir on Nimbus. Cleaning up...")
    +      (.remove-storm! storm-cluster-state corrupt)
    +      (if (instance? LocalFsBlobStore blob-store)
    +        (doseq [blob-key (get-key-list-from-id (:conf nimbus) corrupt)]
    +          (.remove-blobstore-key! storm-cluster-state blob-key)))
    +      )))
    +
    +;;setsup blobstore for all current keys
    +(defn setup-blobstore [nimbus]
    +  (let [storm-cluster-state (:storm-cluster-state nimbus)
    +        blob-store (:blob-store nimbus)
    +        local-list-of-keys (set (get-key-list-from-blob-store blob-store))
    --- End diff --
    
    In `get-key-list-from-blob-store`, you convert to a Seq, then to a vector, here to a set, and below to an ArrayList.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46101477
  
    --- Diff: external/storm-hdfs/src/test/java/org/apache/storm/hdfs/blobstore/BlobStoreTest.java ---
    @@ -0,0 +1,518 @@
    +/**
    + * 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 org.apache.storm.hdfs.blobstore;
    +
    +import backtype.storm.Config;
    +import backtype.storm.blobstore.AtomicOutputStream;
    +import backtype.storm.blobstore.BlobStore;
    +import backtype.storm.blobstore.BlobStoreAclHandler;
    +import backtype.storm.generated.AccessControl;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.generated.AccessControlType;
    +
    +import backtype.storm.security.auth.NimbusPrincipal;
    +import backtype.storm.security.auth.SingleUserPrincipal;
    +import backtype.storm.utils.Utils;
    +import org.apache.commons.io.FileUtils;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hdfs.MiniDFSCluster;
    +import org.junit.After;
    +import org.junit.AfterClass;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.mockito.Mockito;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.URI;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.UUID;
    +import java.util.HashSet;
    +import java.util.Set;
    +import java.util.Iterator;
    +import java.util.Arrays;
    +import java.util.List;
    +import java.util.ArrayList;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertTrue;
    +import static org.junit.Assert.fail;
    +import static org.mockito.Mockito.*;
    +
    +public class BlobStoreTest {
    +  private static final Logger LOG = LoggerFactory.getLogger(BlobStoreTest.class);
    +  protected static MiniDFSCluster dfscluster = null;
    +  protected static Configuration hadoopConf = null;
    +  URI base;
    +  File baseFile;
    +  private static Map conf = new HashMap();
    +  public static final int READ = 0x01;
    +  public static final int WRITE = 0x02;
    +  public static final int ADMIN = 0x04;
    +
    +  @Before
    +  public void init() {
    +    initializeConfigs();
    +    baseFile = new File("/tmp/blob-store-test-"+UUID.randomUUID());
    +    base = baseFile.toURI();
    +  }
    +
    +  @After
    +  public void cleanup() throws IOException {
    +    FileUtils.deleteDirectory(baseFile);
    +  }
    +
    +  @AfterClass
    +  public static void cleanupAfterClass() throws IOException {
    +    if (dfscluster != null) {
    +      dfscluster.shutdown();
    +    }
    +  }
    +
    +  // Method which initializes nimbus admin
    +  public static void initializeConfigs() {
    +    conf.put(Config.NIMBUS_ADMINS,"admin");
    +    conf.put(Config.NIMBUS_SUPERVISOR_USERS,"supervisor");
    +  }
    +
    +  //Gets Nimbus Subject with NimbusPrincipal set on it
    +  public static Subject getNimbusSubject() {
    +    Subject nimbus = new Subject();
    +    nimbus.getPrincipals().add(new NimbusPrincipal());
    +    return nimbus;
    +  }
    +
    +  // Overloading the assertStoreHasExactly method accomodate Subject in order to check for authorization
    +  public static void assertStoreHasExactly(BlobStore store, Subject who, String ... keys)
    +          throws IOException, KeyNotFoundException, AuthorizationException {
    +    Set<String> expected = new HashSet<String>(Arrays.asList(keys));
    +    Set<String> found = new HashSet<String>();
    +    Iterator<String> c = store.listKeys();
    +    while (c.hasNext()) {
    +      String keyName = c.next();
    +      found.add(keyName);
    +    }
    +    Set<String> extra = new HashSet<String>(found);
    +    extra.removeAll(expected);
    +    assertTrue("Found extra keys in the blob store "+extra, extra.isEmpty());
    +    Set<String> missing = new HashSet<String>(expected);
    +    missing.removeAll(found);
    +    assertTrue("Found keys missing from the blob store "+missing, missing.isEmpty());
    +  }
    +
    +  public static void assertStoreHasExactly(BlobStore store, String ... keys)
    +          throws IOException, KeyNotFoundException, AuthorizationException {
    +    assertStoreHasExactly(store, null, keys);
    +  }
    +
    +  // Overloading the readInt method accomodate Subject in order to check for authorization (security turned on)
    +  public static int readInt(BlobStore store, Subject who, String key) throws IOException, KeyNotFoundException, AuthorizationException {
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r44997943
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -927,31 +1133,32 @@
            first ))
     
     (defmethod download-storm-code
    -    :local [conf storm-id master-code-dir supervisor download-lock]
    -    (let [stormroot (supervisor-stormdist-root conf storm-id)]
    -      (locking download-lock
    -            (FileUtils/copyDirectory (File. master-code-dir) (File. stormroot))
    -            (let [classloader (.getContextClassLoader (Thread/currentThread))
    -                  resources-jar (resources-jar)
    -                  url (.getResource classloader RESOURCES-SUBDIR)
    -                  target-dir (str stormroot file-path-separator RESOURCES-SUBDIR)]
    -              (cond
    -               resources-jar
    -               (do
    -                 (log-message "Extracting resources from jar at " resources-jar " to " target-dir)
    -                 (extract-dir-from-jar resources-jar RESOURCES-SUBDIR stormroot))
    -               url
    -               (do
    -                 (log-message "Copying resources at " (URI. (str url)) " to " target-dir)
    -                 (if (= (.getProtocol url) "jar" )
    -                   (extract-dir-from-jar (.getFile (.getJarFileURL (.openConnection url))) RESOURCES-SUBDIR stormroot)
    -                   (FileUtils/copyDirectory (File. (.getPath (URI. (str url)))) (File. target-dir)))
    -                 )
    -               )
    -              )
    -            )))
    -
    -(defmethod mk-code-distributor :local [conf] nil)
    +  :local [conf storm-id master-code-dir localizer]
    +  (let [tmproot (str (supervisor-tmp-dir conf) file-path-separator (uuid))
    +        stormroot (supervisor-stormdist-root conf storm-id)
    +        blob-store (Utils/getNimbusBlobStore conf master-code-dir nil)]
    +    (try
    +      (FileUtils/forceMkdir (File. tmproot))
    +      (.readBlobTo blob-store (master-stormcode-key storm-id) (FileOutputStream. (supervisor-stormcode-path tmproot)) nil)
    +      (.readBlobTo blob-store (master-stormconf-key storm-id) (FileOutputStream. (supervisor-stormconf-path tmproot)) nil)
    +      (finally
    +        (.shutdown blob-store)))
    +    (FileUtils/moveDirectory (File. tmproot) (File. stormroot))
    +    (setup-storm-code-dir conf (read-supervisor-storm-conf conf storm-id) stormroot)
    +    (let [classloader (.getContextClassLoader (Thread/currentThread))
    +          resources-jar (resources-jar)
    +          url (.getResource classloader RESOURCES-SUBDIR)
    +          target-dir (str stormroot file-path-separator RESOURCES-SUBDIR)]
    +      (cond
    +        resources-jar
    --- End diff --
    
    Lots of blank space at the end of this line?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by unsleepy22 <gi...@git.apache.org>.
Github user unsleepy22 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46023394
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobSynchronizer.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * 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.blobstore.BlobStoreUtils;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Set;;
    +
    +/**
    + * Is called periodically and updates the nimbus with blobs based on the state stored inside the zookeeper
    + * for a non leader nimbus trying to be in sync with the operations performed on the leader nimbus.
    + */
    +public class BlobSynchronizer {
    +  private static final Logger LOG = LoggerFactory.getLogger(BlobSynchronizer.class);
    +  private CuratorFramework zkClient;
    +  private Map conf;
    +  private BlobStore blobStore;
    +  private Set<String> blobStoreKeySet = new HashSet<String>();
    +  private Set<String> zookeeperKeySet = new HashSet<String>();
    +  private NimbusInfo nimbusInfo;
    +
    +  public BlobSynchronizer(BlobStore blobStore, Map conf) {
    +    this.blobStore = blobStore;
    +    this.conf = conf;
    +  }
    +
    +  public void setNimbusInfo(NimbusInfo nimbusInfo) {
    +    this.nimbusInfo = nimbusInfo;
    +  }
    +
    +  public void setZookeeperKeySet(Set<String> zookeeperKeySet) {
    +    this.zookeeperKeySet = zookeeperKeySet;
    +  }
    +
    +  public void setBlobStoreKeySet(Set<String> blobStoreKeySet) {
    +    this.blobStoreKeySet = blobStoreKeySet;
    +  }
    +
    +  public Set<String> getBlobStoreKeySet() {
    +    Set<String> keySet = new HashSet<String>();
    +    keySet.addAll(blobStoreKeySet);
    +    return keySet;
    +  }
    +
    +  public Set<String> getZookeeperKeySet() {
    +    Set<String> keySet = new HashSet<String>();
    +    keySet.addAll(zookeeperKeySet);
    +    return keySet;
    +  }
    +
    +  public synchronized void syncBlobs() {
    +    try {
    +    LOG.debug("Sync blobs - blobstore keys {}, zookeeper keys {}",getBlobStoreKeySet(), getZookeeperKeySet());
    --- End diff --
    
    missing indent in try block


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45432788
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStore.java ---
    @@ -0,0 +1,306 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.utils.Utils;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.thrift.TBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.ByteArrayOutputStream;
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.FileNotFoundException;
    +import java.io.InputStream;
    +
    +import java.util.*;
    +
    +import static backtype.storm.blobstore.BlobStoreAclHandler.ADMIN;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.READ;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.WRITE;
    +
    +/**
    + * Provides a local file system backed blob store implementation for Nimbus.
    + */
    +public class LocalFsBlobStore extends BlobStore {
    +  public static final Logger LOG = LoggerFactory.getLogger(LocalFsBlobStore.class);
    +  private static final String DATA_PREFIX = "data_";
    +  private static final String META_PREFIX = "meta_";
    +  protected BlobStoreAclHandler _aclHandler;
    +  private final String BLOBSTORE_SUBTREE = "/blobstore/";
    +  private NimbusInfo nimbusInfo;
    +  private FileBlobStoreImpl fbs;
    +  private Map conf;
    +
    +  @Override
    +  public void prepare(Map conf, String overrideBase, NimbusInfo nimbusInfo) {
    +    this.conf = conf;
    +    this.nimbusInfo = nimbusInfo;
    +     if (overrideBase == null) {
    +      overrideBase = (String)conf.get(Config.BLOBSTORE_DIR);
    +      if (overrideBase == null) {
    +        overrideBase = (String) conf.get(Config.STORM_LOCAL_DIR);
    +      }
    +    }
    +    File baseDir = new File(overrideBase, BASE_BLOBS_DIR_NAME);
    +    try {
    +      fbs = new FileBlobStoreImpl(baseDir, conf);
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +    _aclHandler = new BlobStoreAclHandler(conf);
    +  }
    +
    +  @Override
    +  public AtomicOutputStream createBlob(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException {
    +    LOG.debug("Creating Blob for key {}", key);
    +    validateKey(key);
    +    _aclHandler.normalizeSettableBlobMeta(key, meta, who, READ | WRITE | ADMIN);
    +    BlobStoreAclHandler.validateSettableACLs(key, meta.get_acl());
    +    _aclHandler.hasPermissions(meta.get_acl(), READ | WRITE | ADMIN, who, key);
    +    if (fbs.exists(DATA_PREFIX+key)) {
    +      throw new KeyAlreadyExistsException(key);
    +    }
    +    BlobStoreFileOutputStream mOut = null;
    +    try {
    +      mOut = new BlobStoreFileOutputStream(fbs.write(META_PREFIX+key, true));
    +      mOut.write(Utils.thriftSerialize((TBase) meta));
    --- End diff --
    
    `meta` is already a `TBase`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45427102
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/KeyFilter.java ---
    @@ -0,0 +1,22 @@
    +/**
    + * 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;
    +
    +public interface KeyFilter<R> {
    +  public R filter(String key);
    --- End diff --
    
    Don't need the `public`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45123873
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -1767,6 +1900,169 @@
                                       topology-id
                                       (doto (GetInfoOptions.) (.set_num_err_choice NumErrorsChoice/ALL))))
     
    +      ;;Blobstore implementation code
    +      (^String beginCreateBlob [this
    +                                ^String blob-key
    +                                ^SettableBlobMeta blob-meta]
    +        (let [session-id (uuid)]
    +          (.put (:blob-uploaders nimbus)
    +            session-id
    +            (->> (ReqContext/context)
    +              (.subject)
    +              (.createBlob (:blob-store nimbus) blob-key blob-meta)))
    +          (log-message "Created blob for " blob-key
    +            " with session id " session-id)
    +          (str session-id)))
    +
    +      (^String beginUpdateBlob [this ^String blob-key]
    +        (let [^AtomicOutputStream os (->> (ReqContext/context)
    +                                       (.subject)
    +                                       (.updateBlob (:blob-store nimbus)
    +                                         blob-key))]
    +          (let [session-id (uuid)]
    +            (.put (:blob-uploaders nimbus) session-id os)
    +            (log-message "Created upload session for " blob-key
    +              " with id " session-id)
    +            (str session-id))))
    +
    +      (^void createStateInZookeeper [this ^String blob-key]
    +        (let [storm-cluster-state (:storm-cluster-state nimbus)
    +              blob-store (:blob-store nimbus)
    +              nimbus-host-port-info (:nimbus-host-port-info nimbus)
    +              conf (:conf nimbus)]
    +          (if (instance? LocalFsBlobStore blob-store)
    +              (.setup-blobstore! storm-cluster-state blob-key nimbus-host-port-info (get-version-for-key blob-key nimbus-host-port-info conf)))
    +          (log-debug "Created state in zookeeper" storm-cluster-state blob-store nimbus-host-port-info)))
    +
    +      (^void uploadBlobChunk [this ^String session ^ByteBuffer blob-chunk]
    +        (let [uploaders (:blob-uploaders nimbus)]
    +          (if-let [^AtomicOutputStream os (.get uploaders session)]
    +            (let [chunk-array (.array blob-chunk)
    +                  remaining (.remaining blob-chunk)
    +                  array-offset (.arrayOffset blob-chunk)
    +                  position (.position blob-chunk)]
    +              (.write os chunk-array (+ array-offset position) remaining)
    +              (.put uploaders session os))
    +            (throw-runtime "Blob for session "
    +              session
    +              " does not exist (or timed out)"))))
    +
    +      (^void finishBlobUpload [this ^String session]
    +        (if-let [^AtomicOutputStream os (.get (:blob-uploaders nimbus) session)]
    +          (do
    +            (.close os)
    +            (log-message "Finished uploading blob for session "
    +              session
    +              ". Closing session.")
    +            (.remove (:blob-uploaders nimbus) session))
    +          (throw-runtime "Blob for session "
    +            session
    +            " does not exist (or timed out)")))
    +
    +      (^void cancelBlobUpload [this ^String session]
    +        (if-let [^AtomicOutputStream os (.get (:blob-uploaders nimbus) session)]
    +          (do
    +            (.cancel os)
    +            (log-message "Canceled uploading blob for session "
    +              session
    +              ". Closing session.")
    +            (.remove (:blob-uploaders nimbus) session))
    +          (throw-runtime "Blob for session "
    +            session
    +            " does not exist (or timed out)")))
    +
    +      (^ReadableBlobMeta getBlobMeta [this ^String blob-key]
    +        (let [^ReadableBlobMeta ret (->> (ReqContext/context)
    +                                      (.subject)
    +                                      (.getBlobMeta (:blob-store nimbus)
    +                                        blob-key))]
    +          ret))
    +
    +      (^void setBlobMeta [this ^String blob-key ^SettableBlobMeta blob-meta]
    +        (->> (ReqContext/context)
    +          (.subject)
    +          (.setBlobMeta (:blob-store nimbus) blob-key blob-meta)))
    +
    +      (^BeginDownloadResult beginBlobDownload [this ^String blob-key]
    +        (let [^InputStreamWithMeta is (->> (ReqContext/context)
    +                                        (.subject)
    +                                        (.getBlob (:blob-store nimbus)
    +                                          blob-key))]
    +          (let [session-id (uuid)
    +                ret (BeginDownloadResult. (.getVersion is) (str session-id))]
    +            (.set_data_size ret (.getFileLength is))
    +            (.put (:blob-downloaders nimbus) session-id (BufferInputStream. is ^Integer (Utils/getInt (conf STORM-BLOBSTORE-INPUTSTREAM-BUFFER-SIZE-BYTES) (int 65536))))
    +            (log-message "Created download session for " blob-key
    +              " with id " session-id)
    +            ret)))
    +
    +      (^ByteBuffer downloadBlobChunk [this ^String session]
    +        (let [downloaders (:blob-downloaders nimbus)
    +              ^BufferInputStream is (.get downloaders session)]
    +          (when-not is
    +            (throw (RuntimeException.
    +                     "Could not find input stream for session " session)))
    +          (let [ret (.read is)]
    +            (.put downloaders session is)
    +            (when (empty? ret)
    +              (.close is)
    +              (.remove downloaders session))
    +            (log-debug "Sending " (alength ret) " bytes")
    +            (ByteBuffer/wrap ret))))
    +
    +      (^void deleteBlob [this ^String blob-key]
    +        (let [subject (->> (ReqContext/context)
    +                           (.subject))]
    +        (.deleteBlob (:blob-store nimbus) blob-key subject)
    +        (when (instance? LocalFsBlobStore blob-store)
    +          (.remove-blobstore-key! (:storm-cluster-state nimbus) blob-key)
    +          (.remove-key-version! (:storm-cluster-state nimbus) blob-key))
    +        (log-message "Deleted blob for key " blob-key)))
    +
    +      (^ListBlobsResult listBlobs [this ^String session]
    +        (let [listers (:blob-listers nimbus)
    +              ^Iterator keys-it (if (clojure.string/blank? session)
    +                                  (->> (ReqContext/context)
    +                                    (.subject)
    +                                    (.listKeys (:blob-store nimbus)))
    +                                  (.get listers session))
    +              _ (or keys-it (throw-runtime "Blob list for session "
    +                              session
    +                              " does not exist (or timed out)"))
    +
    +              ;; Create a new session id if the user gave an empty session string.
    +              ;; This is the use case when the user wishes to list blobs
    +              ;; starting from the beginning.
    +              session (if (clojure.string/blank? session)
    +                        (let [new-session (uuid)]
    +                          (log-message "Creating new session for downloading list " new-session)
    +                          new-session)
    +                        session)]
    +          (if-not (.hasNext keys-it)
    +            (do
    +              (.remove listers session)
    +              (log-message "No more blobs to list for session " session)
    +              ;; A blank result communicates that there are no more blobs.
    +              (ListBlobsResult. (ArrayList. 0) (str session)))
    +            (let [^List list-chunk (->> keys-it
    +                                     (iterator-seq)
    +                                     (take 100) ;; Limit to next 100 keys
    +                                     (ArrayList.))
    +                  _ (log-message session " downloading " (.size list-chunk) " entries")]
    --- End diff --
    
    This can be moved outside the `let` binding, before the `.put`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the pull request:

    https://github.com/apache/storm/pull/845#issuecomment-153448299
  
    @ptgoetz I know you want to be careful about what gets merged in, and if you insist we will go through the IP clearance process, but if we can avoid it I really would prefer to do so.  We have not had any issues in the past with large commits, like when we contributed the security code to storm which followed essentially the same process as this code. On my end it is probably going to take longer than the 72 hour waiting period to track down the right people to get signatures and they will all ask me why we have to do this when my team has blanket approval to contribute to storm.
    
    Like I said if you insist on getting IP clearance we will do it, but all it is just going to do is add more pain for me which I really would like to avoid if I can.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
GitHub user redsanket reopened a pull request:

    https://github.com/apache/storm/pull/845

    [STORM-876] Blobstore API

    Please take a look at the blobstore API. The design docs have been uploaded earlier under the same JIRA. Currently, the blobstore API also works with Nimbus-HA. Apart from the unit tests written for blobstore and nimbus leader election, I have performed few manual tests. Having the PR early looking for suggestions and any missed failover scenarios while the work continues on other additional integration tests and better interface. The manual tests that have been performed are
    1. CRUD operations for the blobstore API. Tested for the Local and Hdfs blobstore operations to work with Nimbus-HA and whether the other nimbodes mirror the operations performed on master nimbus.
    2. Brought up 3 nimbodes and launched a word count topology. Killed the leader daemon and checked whether other non-leader takes up the leadership and continues to run.
    3. Tested whether topology waits for replication to take place before it is launched. For hdfs blobstore the blobstore.replication should be set greater than or equal to topology.min.replication to work effectively.
    4. The blobstore with nimbus-ha does not guarantee updates to sync effectively in the case of failover.
    
    Follow up code to be written.
    1. Integration test for blobstore with nimbus-ha
    2. Move the sync thread in nimbus to blobstore. Waiting on pacemaker push back as it uses some of its code and write the callback code for blobstore

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/redsanket/storm blobstore

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/storm/pull/845.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #845
    
----
commit 6e1ee30ed6e142033a8b3f264918806630da1bf1
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-02T19:59:13Z

    Squashed commit of the following:
    
    commit 3403a6da661f5ff002cf5e4571853f9b95d8e3c8
    Author: Sanket <sc...@untilservice-lm>
    Date:   Mon Nov 2 13:38:14 2015 -0600
    
        resolved unit test errors caused due to merge commits
    
    commit 898c1d64b5dcbcca2d1efc9a78e731316c1292c0
    Merge: 9909a50 8ba776b
    Author: Sanket <sc...@untilservice-lm>
    Date:   Mon Nov 2 11:41:13 2015 -0600
    
        resolved merge conflicts
    
    commit 9909a50a8b1037e16b21f45ee88c8237f33bd7b8
    Author: Sanket <sc...@untilservice-lm>
    Date:   Mon Nov 2 10:57:29 2015 -0600
    
        cleaned-up debug statements
    
    commit 3f2e1796a74799702113303e896d353ce4218c29
    Author: Sanket <sc...@untilservice-lm>
    Date:   Fri Oct 30 23:25:21 2015 -0500
    
        zookeeper state code and code clean up
    
    commit ac4db652aaad8f3ec71aca50eb31579bf90bd887
    Author: Sanket <sc...@untilservice-lm>
    Date:   Thu Oct 29 11:26:17 2015 -0500
    
        made changes to the sync blob code to fix race condition
    
    commit c96ffcaac5d247da1c77a1e1699b2d7b7f76ebf5
    Author: Sanket <sc...@untilservice-lm>
    Date:   Tue Oct 27 13:38:17 2015 -0500
    
        resolved unit test errors
    
    commit f8f7ebab8cb636a296d84b7e719c368e988063c8
    Author: Sanket <sc...@untilservice-lm>
    Date:   Mon Oct 26 22:51:11 2015 -0500
    
        added callback function
    
    commit c5f02cd54ef50039a6f4dde8f073e839b9ef7b3a
    Author: Sanket <sc...@untilservice-lm>
    Date:   Fri Oct 23 16:20:41 2015 -0500
    
        merge commit, modified unit tests to suit blob-store
    
    commit fdc77d54054470fdaa419dbedaa0d69fdf1f63ce
    Merge: eb0ea1e f75fdde
    Author: Sanket <sc...@untilservice-lm>
    Date:   Wed Oct 21 13:28:46 2015 -0500
    
        modified blobstore to accomodate nimbus ha
    
    commit eb0ea1e167ebfee94e3a0911f05c2102cdeb77b8
    Author: Sanket <sc...@untilservice-lm>
    Date:   Wed Oct 21 13:08:22 2015 -0500
    
        integrated blobstore code
    
    commit f25e7ba8139b0e2ed56ab532339d0b15b4563c6e
    Author: Sanket <sc...@untilservice-lm>
    Date:   Thu Oct 15 19:06:52 2015 -0500
    
        initial commit
    
    commit ba74039b93ee34067e2511e280a50e3958dc8741
    Author: Sanket <sc...@untilservice-lm>
    Date:   Thu Oct 8 15:04:42 2015 -0500
    
        modified thrift files to include blobstore related classes
    
    commit 73859f70e21e4acde0b3b906b53786c7d2b7a5c5
    Author: Sanket <sc...@untilservice-lm>
    Date:   Thu Oct 8 15:01:46 2015 -0500
    
        integrating blobstore and making sure it builds

commit 2506665b02d8808efd20760f1a11e99239518496
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-02T20:34:09Z

    removed log message and fixed a unit test

commit e8093acbecdfc8893bd9ac571ee8e48f3d610e80
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-03T03:32:33Z

    removed date change generated files

commit 04458b8f8d3ecb022995281630c130da30dd7b28
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-03T04:13:01Z

    indentation edit

commit a26bdf8efd94d26ba65489491594a712381373c7
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-03T04:16:06Z

    indentation edit

commit e2fdb5e9f01cf0e58ed102839ff9a01e757cb290
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-03T05:05:33Z

    removed a commented method

commit 73a5d2f9fcab0063cabf39ff9e7ba4486e1e9070
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-03T05:08:05Z

    removed an unnecessary commit

commit 435913a8325cd6c6085beba90afebae97bc29e3e
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-03T15:19:30Z

    removed few more date change generated files

commit 87c89d561074e478a30f2ba611b068e2a05cfb91
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-03T18:29:58Z

    indentation edits and formatting issues

commit 8825b570d2e63eefdb04b2b72a681c22adaefb20
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-03T21:01:24Z

    moved the createInZookeeper to command line and got rid of few indentation errors and space errors

commit 114a2c14bad8ab3695b8f18c72813d343e56c8e9
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-04T02:07:05Z

    addressed few more issues on indentation, formatting and documentation

commit a1165a08237d6e639f3403ca340555be762cf54a
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-04T02:12:22Z

    addressed issues related to code optimization and removed comments from storm thrift as the design docs explain them

commit 45837ecf26ad99019c890757d68f1a9fa6bd624b
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-04T15:49:17Z

    removed date change generated thrift files

commit f1e0fbbada106a2b3adc9e0c9a0fed4ca01aa0a7
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-04T16:12:59Z

    fixed to resolve single class imports

commit f89d272fa7646e4715f209c46b581311fc3150f4
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-04T20:07:07Z

    Added config validation map for blobstore map

commit 73f02571728c88c37df14a5cbc78c7f0dd4541d3
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-04T23:16:47Z

    removed Blob Replication struct

commit 27aad4137e58f232f7ef8e69f44de7c79952d50e
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-05T00:40:53Z

    resolved few more indentation, spacing errors as per review

commit 8691b7c0fbaa94ec5428844e008e08b434a9056c
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-05T16:30:09Z

    made some changes to the debug logs

commit 650e386130a0506b909327dfc9a2ce3a00644901
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-07T16:14:46Z

    added code to handle update scenarios by implementing a version generator

commit 993c851e7ebbc4092f4b65d9b55ce1fe682af0c7
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-08T05:09:41Z

    added few debug statements, refactoring and removed a bug found during updating blob

commit 19d2b18ee99dfdd33741adae1e742167684d2f2a
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-08T21:55:03Z

    removed few debug comments, refactoring and handling few more failure scenarios

commit 5d19faf59e76d5321ed0bb693189420e0debc635
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-08T22:28:10Z

    merge commit

commit 4a03bcd607de2fd894eb776227ba7c52ff57a898
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-08T22:40:01Z

    added apache licenses and resolved a unchecked merge commit

commit 115e25bbdea1e1ef671a42854f33a837953a24e1
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-08T22:41:24Z

    resolved missed merge commit

commit a49b47d132c8c9fe8f3e546d2a3b7a0730f2c732
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-08T22:45:49Z

    resolved arity exception caused during merge

commit 1e5e41f7b332d25c1fc793827549608de5f864bd
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-09T03:19:21Z

    resolved a unit test error caused by merge

commit 32dff50a4d351477ec1854adc5c083fd48dc3f0f
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-09T15:45:41Z

    code clean up and exclude build directory from checking for license headers

commit bae22c841f326d09125df6ab2492a1e6dbf0d12b
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-09T17:27:46Z

    removed unused import

commit 32e5e791e8abc554e4413d38c4c229d797c58568
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-09T18:55:46Z

    change to error log statements

commit 8efd0ba9fb1307c38f3481f7ba2b0dd2aed90a17
Author: Sanket <sc...@untilservice-lm>
Date:   2015-11-10T19:11:38Z

    merge commit

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by unsleepy22 <gi...@git.apache.org>.
Github user unsleepy22 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46018454
  
    --- Diff: external/storm-hdfs/src/test/java/org/apache/storm/hdfs/blobstore/BlobStoreTest.java ---
    @@ -0,0 +1,518 @@
    +/**
    + * 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 org.apache.storm.hdfs.blobstore;
    +
    +import backtype.storm.Config;
    +import backtype.storm.blobstore.AtomicOutputStream;
    +import backtype.storm.blobstore.BlobStore;
    +import backtype.storm.blobstore.BlobStoreAclHandler;
    +import backtype.storm.generated.AccessControl;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.generated.AccessControlType;
    +
    +import backtype.storm.security.auth.NimbusPrincipal;
    +import backtype.storm.security.auth.SingleUserPrincipal;
    +import backtype.storm.utils.Utils;
    +import org.apache.commons.io.FileUtils;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hdfs.MiniDFSCluster;
    +import org.junit.After;
    +import org.junit.AfterClass;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.mockito.Mockito;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.URI;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.UUID;
    +import java.util.HashSet;
    +import java.util.Set;
    +import java.util.Iterator;
    +import java.util.Arrays;
    +import java.util.List;
    +import java.util.ArrayList;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertTrue;
    +import static org.junit.Assert.fail;
    +import static org.mockito.Mockito.*;
    +
    +public class BlobStoreTest {
    +  private static final Logger LOG = LoggerFactory.getLogger(BlobStoreTest.class);
    +  protected static MiniDFSCluster dfscluster = null;
    +  protected static Configuration hadoopConf = null;
    +  URI base;
    +  File baseFile;
    +  private static Map conf = new HashMap();
    +  public static final int READ = 0x01;
    +  public static final int WRITE = 0x02;
    +  public static final int ADMIN = 0x04;
    +
    +  @Before
    +  public void init() {
    +    initializeConfigs();
    +    baseFile = new File("/tmp/blob-store-test-"+UUID.randomUUID());
    +    base = baseFile.toURI();
    +  }
    +
    +  @After
    +  public void cleanup() throws IOException {
    +    FileUtils.deleteDirectory(baseFile);
    +  }
    +
    +  @AfterClass
    +  public static void cleanupAfterClass() throws IOException {
    +    if (dfscluster != null) {
    +      dfscluster.shutdown();
    +    }
    +  }
    +
    +  // Method which initializes nimbus admin
    +  public static void initializeConfigs() {
    +    conf.put(Config.NIMBUS_ADMINS,"admin");
    +    conf.put(Config.NIMBUS_SUPERVISOR_USERS,"supervisor");
    +  }
    +
    +  //Gets Nimbus Subject with NimbusPrincipal set on it
    +  public static Subject getNimbusSubject() {
    +    Subject nimbus = new Subject();
    +    nimbus.getPrincipals().add(new NimbusPrincipal());
    +    return nimbus;
    +  }
    +
    +  // Overloading the assertStoreHasExactly method accomodate Subject in order to check for authorization
    +  public static void assertStoreHasExactly(BlobStore store, Subject who, String ... keys)
    +          throws IOException, KeyNotFoundException, AuthorizationException {
    +    Set<String> expected = new HashSet<String>(Arrays.asList(keys));
    +    Set<String> found = new HashSet<String>();
    +    Iterator<String> c = store.listKeys();
    +    while (c.hasNext()) {
    +      String keyName = c.next();
    +      found.add(keyName);
    +    }
    +    Set<String> extra = new HashSet<String>(found);
    +    extra.removeAll(expected);
    +    assertTrue("Found extra keys in the blob store "+extra, extra.isEmpty());
    +    Set<String> missing = new HashSet<String>(expected);
    +    missing.removeAll(found);
    +    assertTrue("Found keys missing from the blob store "+missing, missing.isEmpty());
    +  }
    +
    +  public static void assertStoreHasExactly(BlobStore store, String ... keys)
    +          throws IOException, KeyNotFoundException, AuthorizationException {
    +    assertStoreHasExactly(store, null, keys);
    +  }
    +
    +  // Overloading the readInt method accomodate Subject in order to check for authorization (security turned on)
    +  public static int readInt(BlobStore store, Subject who, String key) throws IOException, KeyNotFoundException, AuthorizationException {
    --- End diff --
    
    new line for throws statement, same for all below.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by kishorvpatil <gi...@git.apache.org>.
Github user kishorvpatil commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43821366
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStoreAclHandler.java ---
    @@ -0,0 +1,382 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.AccessControl;
    +import backtype.storm.generated.AccessControlType;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.security.auth.AuthUtils;
    +import backtype.storm.security.auth.IPrincipalToLocal;
    +import backtype.storm.security.auth.NimbusPrincipal;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.security.Principal;
    +import java.util.*;
    --- End diff --
    
    explicit import


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45112555
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -390,53 +444,98 @@
           [(.getNodeId slot) (.getPort slot)]
           )))
     
    +(defn- get-version-for-key [key nimbus-host-port-info conf]
    +  (let [version (KeyVersion. key nimbus-host-port-info)]
    +    (.getKeyVersion version conf)))
    +
    +(defn get-key-seq-from-blob-store [blob-store]
    +  (let [key-iter (.listKeys blob-store nimbus-subject)]
    +    (iterator-seq key-iter)))
    +
     (defn- setup-storm-code [nimbus conf storm-id tmp-jar-location storm-conf topology]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -   (log-message "nimbus file location:" stormroot)
    -   (FileUtils/forceMkdir (File. stormroot))
    -   (FileUtils/cleanDirectory (File. stormroot))
    -   (setup-jar conf tmp-jar-location stormroot)
    -   (FileUtils/writeByteArrayToFile (File. (master-stormcode-path stormroot)) (Utils/serialize topology))
    -   (FileUtils/writeByteArrayToFile (File. (master-stormconf-path stormroot)) (Utils/toCompressedJsonConf storm-conf))
    -   (if (:code-distributor nimbus) (.upload (:code-distributor nimbus) stormroot storm-id))
    -   ))
    +  (let [subject (get-subject)
    +        storm-cluster-state (:storm-cluster-state nimbus)
    +        blob-store (:blob-store nimbus)
    +        jar-key (master-stormjar-key storm-id)
    +        code-key (master-stormcode-key storm-id)
    +        conf-key (master-stormconf-key storm-id)
    +        nimbus-host-port-info (:nimbus-host-port-info nimbus)]
    +    (when tmp-jar-location ;;in local mode there is no jar
    +      (.createBlob blob-store jar-key (FileInputStream. tmp-jar-location) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +      (if (instance? LocalFsBlobStore blob-store)
    +        (.setup-blobstore! storm-cluster-state jar-key nimbus-host-port-info (get-version-for-key jar-key nimbus-host-port-info conf))))
    +    (.createBlob blob-store conf-key (Utils/toCompressedJsonConf storm-conf) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state conf-key nimbus-host-port-info (get-version-for-key conf-key nimbus-host-port-info conf)))
    +    (.createBlob blob-store code-key (Utils/serialize topology) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state code-key nimbus-host-port-info (get-version-for-key code-key nimbus-host-port-info conf)))))
    +
    +(defn- read-storm-topology [storm-id blob-store]
    +  (Utils/deserialize
    +    (.readBlob blob-store (master-stormcode-key storm-id) (get-subject)) StormTopology))
    +
    +(defn- get-blob-replication-count [blob-key nimbus]
    +  (if (:blob-store nimbus)
    +          (-> (:blob-store nimbus)
    +            (.getBlobReplication  blob-key nimbus-subject))))
     
     (defn- wait-for-desired-code-replication [nimbus conf storm-id]
       (let [min-replication-count (conf TOPOLOGY-MIN-REPLICATION-COUNT)
             max-replication-wait-time (conf TOPOLOGY-MAX-REPLICATION-WAIT-TIME-SEC)
    -        total-wait-time (atom 0)
    -        current-replication-count (atom (if (:code-distributor nimbus) (.getReplicationCount (:code-distributor nimbus) storm-id) 0))]
    -  (if (:code-distributor nimbus)
    -    (while (and (> min-replication-count @current-replication-count)
    -             (or (= -1 max-replication-wait-time)
    -               (< @total-wait-time max-replication-wait-time)))
    +        current-replication-count-jar (if (not (local-mode? conf))
    +                                        (atom (get-blob-replication-count (master-stormjar-key storm-id) nimbus))
    +                                        (atom min-replication-count))
    +        current-replication-count-code (atom (get-blob-replication-count (master-stormcode-key storm-id) nimbus))
    +        current-replication-count-conf (atom (get-blob-replication-count (master-stormconf-key storm-id) nimbus))
    +        total-wait-time (atom 0)]
    +    (if (:blob-store nimbus)
    +      (while (and
    +               (or (> min-replication-count @current-replication-count-jar)
    +                   (> min-replication-count @current-replication-count-code)
    +                   (> min-replication-count @current-replication-count-conf))
    +               (or (= -1 max-replication-wait-time)
    +                   (< @total-wait-time max-replication-wait-time)))
             (sleep-secs 1)
             (log-debug "waiting for desired replication to be achieved.
               min-replication-count = " min-replication-count  " max-replication-wait-time = " max-replication-wait-time
    -          "current-replication-count = " @current-replication-count " total-wait-time " @total-wait-time)
    +          (if (not (local-mode? conf))"current-replication-count for jar key = " @current-replication-count-jar)
    +          "current-replication-count for code key = " @current-replication-count-code
    +          "current-replication-count for conf key = " @current-replication-count-conf
    +          " total-wait-time " @total-wait-time)
             (swap! total-wait-time inc)
    -        (reset! current-replication-count  (.getReplicationCount (:code-distributor nimbus) storm-id))))
    -  (if (< min-replication-count @current-replication-count)
    -    (log-message "desired replication count "  min-replication-count " achieved,
    -      current-replication-count" @current-replication-count)
    -    (log-message "desired replication count of "  min-replication-count " not achieved but we have hit the max wait time "
    -      max-replication-wait-time " so moving on with replication count = " @current-replication-count)
    -    )))
    -
    -(defn- read-storm-topology [conf storm-id]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -    (Utils/deserialize
    -      (FileUtils/readFileToByteArray
    -        (File. (master-stormcode-path stormroot))
    -        ) StormTopology)))
    +        (if (not (local-mode? conf))
    +          (reset! current-replication-count-conf  (get-blob-replication-count (master-stormconf-key storm-id))))
    +        (reset! current-replication-count-code  (get-blob-replication-count (master-stormcode-key storm-id)))
    +        (reset! current-replication-count-jar  (get-blob-replication-count (master-stormjar-key storm-id)))))
    --- End diff --
    
    I think I had nimbus before, may be it was caused by merge issue, i do not really know as to how did it compile


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45432533
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStoreAclHandler.java ---
    @@ -0,0 +1,401 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.AccessControl;
    +import backtype.storm.generated.AccessControlType;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.security.auth.AuthUtils;
    +import backtype.storm.security.auth.IPrincipalToLocal;
    +import backtype.storm.security.auth.NimbusPrincipal;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.security.Principal;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Provides common handling of acls for Blobstores.
    + * Also contains some static utility functions related to Blobstores.
    + */
    +public class BlobStoreAclHandler {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStoreAclHandler.class);
    +  private final IPrincipalToLocal _ptol;
    +
    +  public static final int READ = 0x01;
    +  public static final int WRITE = 0x02;
    +  public static final int ADMIN = 0x04;
    +  public static final List<AccessControl> WORLD_EVERYTHING =
    +      Arrays.asList(new AccessControl(AccessControlType.OTHER, READ | WRITE | ADMIN));
    +  public static final List<AccessControl> DEFAULT = new ArrayList<AccessControl>();
    +  private Set<String> _supervisors;
    +  private Set<String> _admins;
    +
    +  public BlobStoreAclHandler(Map conf) {
    +    _ptol = AuthUtils.GetPrincipalToLocalPlugin(conf);
    +    _supervisors = new HashSet<String>();
    +    _admins = new HashSet<String>();
    +    if (conf.containsKey(Config.NIMBUS_SUPERVISOR_USERS)) {
    +      _supervisors.addAll((List<String>)conf.get(Config.NIMBUS_SUPERVISOR_USERS));
    +    }
    +    if (conf.containsKey(Config.NIMBUS_ADMINS)) {
    +      _admins.addAll((List<String>)conf.get(Config.NIMBUS_ADMINS));
    +    }
    +  }
    +
    +  private static AccessControlType parseACLType(String type) {
    +    if ("other".equalsIgnoreCase(type) || "o".equalsIgnoreCase(type)) {
    +      return AccessControlType.OTHER;
    +    } else if ("user".equalsIgnoreCase(type) || "u".equalsIgnoreCase(type)) {
    +      return AccessControlType.USER;
    +    }
    +    throw new IllegalArgumentException(type+" is not a valid access control type");
    +  }
    +
    +  private static int parseAccess(String access) {
    +    int ret = 0;
    +    for (char c: access.toCharArray()) {
    +      if ('r' == c) {
    +        ret = ret | READ;
    +      } else if ('w' == c) {
    +        ret = ret | WRITE;
    +      } else if ('a' == c) {
    +        ret = ret | ADMIN;
    +      } else if ('-' == c) {
    +        //ignored
    +      } else {
    +        throw new IllegalArgumentException("");
    +      }
    +    }
    +    return ret;
    +  }
    +
    +  public static AccessControl parseAccessControl(String str) {
    +    String[] parts = str.split(":");
    +    String type = "other";
    +    String name = "";
    +    String access = "-";
    +    if (parts.length > 3) {
    +      throw new IllegalArgumentException("Don't know how to parse "+str+" into an ACL value");
    +    } else if (parts.length == 1) {
    +      type = "other";
    +      name = "";
    +      access = parts[0];
    +    } else if (parts.length == 2) {
    +      type = "user";
    +      name = parts[0];
    +      access = parts[1];
    +    } else if (parts.length == 3) {
    +      type = parts[0];
    +      name = parts[1];
    +      access = parts[2];
    +    }
    +    AccessControl ret = new AccessControl();
    +    ret.set_type(parseACLType(type));
    +    ret.set_name(name);
    +    ret.set_access(parseAccess(access));
    +    return ret;
    +  }
    +
    +  private static String accessToString(int access) {
    +    StringBuffer ret = new StringBuffer();
    +    ret.append(((access & READ) > 0) ? "r" : "-");
    +    ret.append(((access & WRITE) > 0) ? "w" : "-");
    +    ret.append(((access & ADMIN) > 0) ? "a" : "-");
    +    return ret.toString();
    +  }
    +
    +  public static String accessControlToString(AccessControl ac) {
    +    StringBuffer ret = new StringBuffer();
    +    switch(ac.get_type()) {
    +      case OTHER:
    +        ret.append("o");
    +        break;
    +      case USER:
    +        ret.append("u");
    +        break;
    +      default:
    +        throw new IllegalArgumentException("Ahh don't know what a type of "+ac.get_type()+" means ");
    +    }
    +    ret.append(":");
    +    if (ac.is_set_name()) {
    +      ret.append(ac.get_name());
    +    }
    +    ret.append(":");
    +    ret.append(accessToString(ac.get_access()));
    +    return ret.toString();
    +  }
    +
    +  public static void validateSettableACLs(String key, List<AccessControl> acls) throws AuthorizationException {
    +    Set<String> aclUsers = new HashSet<>();
    +    List<String> duplicateUsers = new ArrayList<>();
    +    for (AccessControl acl : acls) {
    +      String aclUser = acl.get_name();
    +      if (aclUser != null && !aclUser.isEmpty() && !aclUsers.add(aclUser)) {
    +        LOG.error("'{}' user can't appear more than once in the ACLs", aclUser);
    +        duplicateUsers.add(aclUser);
    +      }
    +    }
    +    if (duplicateUsers.size() > 0) {
    +      String errorMessage  = "user " + Arrays.toString(duplicateUsers.toArray())
    +          + " can't appear more than once in the ACLs for key [" + key +"].";
    +      throw new AuthorizationException(errorMessage);
    +    }
    +  }
    +
    +  private Set<String> constructUserFromPrincipals(Subject who) {
    +    Set<String> user = new HashSet<String>();
    +    if (who != null) {
    +      for (Principal p : who.getPrincipals()) {
    +        user.add(_ptol.toLocal(p));
    +      }
    +    }
    +    return user;
    +  }
    +
    +  private boolean isAdmin(Subject who) {
    +    Set<String> user = constructUserFromPrincipals(who);
    +    for (String u : user) {
    +      if (_admins.contains(u)) {
    +        return true;
    +      }
    +    }
    +    return false;
    +  }
    +
    +  private boolean isReadOperation(int operation) {
    +    if (operation == 1) {
    +      return true;
    +    }
    +    return false;
    +  }
    +
    +  private boolean isSupervisor(Subject who, int operation) {
    +    Set<String> user = constructUserFromPrincipals(who);
    +    if (isReadOperation(operation)) {
    +      for (String u : user) {
    +        if (_supervisors.contains(u)) {
    +          return true;
    +        }
    +      }
    +    }
    +    return false;
    +  }
    +
    +  private boolean isNimbus(Subject who) {
    +    Set<Principal> principals = null;
    +    boolean isNimbusInstance = false;
    +    if (who != null) {
    +      principals = who.getPrincipals();
    +      for (Principal principal : principals) {
    +        if (principal instanceof NimbusPrincipal) {
    +          isNimbusInstance = true;
    +        }
    +      }
    +    }
    +    return isNimbusInstance;
    +  }
    +
    +  public boolean checkForValidUsers(Subject who, int mask) {
    +    if (isNimbus(who) || isAdmin(who) || isSupervisor(who,mask)) {
    +      return true;
    +    }
    +    return false;
    +  }
    +
    +  /**
    +   * The user should be able to see the metadata if and only if they have any of READ, WRITE, or ADMIN
    +   */
    +  public void validateUserCanReadMeta(List<AccessControl> acl, Subject who, String key) throws AuthorizationException {
    +    hasAnyPermissions(acl, (READ|WRITE|ADMIN), who, key);
    +  }
    +
    +  /**
    +   * Validates if the user has any of the permissions
    +   * mentioned in the mask.
    +   * @param acl ACL for the key.
    +   * @param mask mask holds the cummulative value of
    +   * READ = 1, WRITE = 2 or ADMIN = 4 permissions.
    +   * mask = 1 implies READ privilege.
    +   * mask = 5 implies READ and ADMIN privileges.
    +   * @param who Is the user against whom the permissions
    +   * are validated for a key using the ACL and the mask.
    +   * @param key Key used to identify the blob.
    +   * @throws AuthorizationException
    +   */
    +  public void hasAnyPermissions(List<AccessControl> acl, int mask, Subject who, String key) throws AuthorizationException {
    +    Set<String> user = constructUserFromPrincipals(who);
    +    LOG.debug("user {}", user);
    +    if (checkForValidUsers(who, mask)) {
    +     return;
    +    }
    +    for (AccessControl ac : acl) {
    +      int allowed = getAllowed(ac, user);
    +      LOG.debug(" user: {} allowed: {} key: {}", user, allowed, key);
    +      if ((allowed & mask) > 0) {
    +        return;
    +      }
    +    }
    +    throw new AuthorizationException(
    +            user + " does not have access to " + key);
    +  }
    +
    +  /**
    +   * Validates if the user has atleast the set of permissions
    +   * mentioned in the mask.
    +   * @param acl ACL for the key.
    +   * @param mask mask holds the cummulative value of
    +   * READ = 1, WRITE = 2 or ADMIN = 4 permissions.
    +   * mask = 1 implies READ privilege.
    +   * mask = 5 implies READ and ADMIN privileges.
    +   * @param who Is the user against whom the permissions.
    +   * are validated for a key using the ACL and the mask.
    +   * @param key Key used to identify the blob.
    +   * @throws AuthorizationException
    +   */
    +  public void hasPermissions(List<AccessControl> acl, int mask, Subject who, String key) throws AuthorizationException {
    +    Set<String> user = constructUserFromPrincipals(who);
    +    LOG.debug("user {}", user);
    +    if (checkForValidUsers(who, mask)) {
    +      return;
    +    }
    +    for (AccessControl ac : acl) {
    +      int allowed = getAllowed(ac, user);
    +      mask = ~allowed & mask;
    +      LOG.debug(" user: {} allowed: {} disallowed: {} key: {}", user, allowed, mask, key);
    +    }
    +    if (mask == 0) {
    +      return;
    +    }
    +    throw new AuthorizationException(
    +            user + " does not have " + namedPerms(mask) + " access to " + key);
    +  }
    +
    +  public void normalizeSettableBlobMeta(String key, SettableBlobMeta meta, Subject who, int opMask) {
    +    meta.set_acl(normalizeSettableACLs(key, meta.get_acl(), who, opMask));
    +  }
    +
    +  private String namedPerms(int mask) {
    +    StringBuffer b = new StringBuffer();
    +    b.append("[");
    +    if ((mask & READ) > 0) {
    +      b.append("READ ");
    +    }
    +    if ((mask & WRITE) > 0) {
    +      b.append("WRITE ");
    +    }
    +    if ((mask & ADMIN) > 0) {
    +      b.append("ADMIN ");
    +    }
    +    b.append("]");
    +    return b.toString();
    +  }
    +
    +  private int getAllowed(AccessControl ac, Set<String> users) {
    +    switch (ac.get_type()) {
    +      case OTHER:
    +        return ac.get_access();
    +      case USER:
    +        if (users.contains(ac.get_name())) {
    +          return ac.get_access();
    +        }
    +        return 0;
    +      default:
    +        return 0;
    +    }
    +  }
    +
    +  private List<AccessControl> removeBadACLs(List<AccessControl> accessControls) {
    +    List<AccessControl> resultAcl = new ArrayList<AccessControl>();
    +    for (AccessControl control : accessControls) {
    +      if(control.get_type().equals(AccessControlType.OTHER) && (control.get_access() == 0 )) {
    +        LOG.debug("Removing invalid blobstore world ACL " +
    +            BlobStoreAclHandler.accessControlToString(control));
    +        continue;
    +      }
    +      resultAcl.add(control);
    +    }
    +    return resultAcl;
    +  }
    +
    +  private final List<AccessControl> normalizeSettableACLs(String key, List<AccessControl> acls, Subject who,
    +                                                    int opMask) {
    +    List<AccessControl> cleanAcls = removeBadACLs(acls);
    +    Set<String> userNames = getUserNamesFromSubject(who);
    +    for (String user : userNames) {
    +      fixACLsForUser(cleanAcls, user, opMask);
    +    }
    +    if ((who == null || userNames.isEmpty()) && !worldEverything(acls)) {
    +        cleanAcls.addAll(BlobStoreAclHandler.WORLD_EVERYTHING);
    --- End diff --
    
    check indentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45261252
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -454,10 +448,79 @@
           (shutdown-worker supervisor id))
         ))
     
    +(defn get-blob-localname
    +  "Given the blob information either gets the localname field if it exists,
    +  else routines the default value passed in."
    +  [blob-info defaultValue]
    +  (or (get blob-info "localname") defaultValue))
    +
    +(defn should-uncompress-blob?
    +  "Given the blob information returns the value of the uncompress field, handling it either being
    +  a string or a boolean value, or if it's not specified then returns false"
    +  [blob-info]
    +  (Boolean. (get blob-info "uncompress")))
    +
    +(defn remove-blob-references
    +  "Remove a reference to a blob when its no longer needed."
    +  [localizer storm-id conf]
    +  (let [storm-conf (read-supervisor-storm-conf conf storm-id)
    +        blobstore-map (storm-conf TOPOLOGY-BLOBSTORE-MAP)
    +        user (storm-conf TOPOLOGY-SUBMITTER-USER)
    +        topo-name (storm-conf TOPOLOGY-NAME)]
    +    (if blobstore-map
    +      (doseq [[k, v] blobstore-map]
    +        (.removeBlobReference localizer
    +          k
    +          user
    +          topo-name
    +          (should-uncompress-blob? v))))))
    +
    +(defn blobstore-map-to-localresources
    +  "Returns a list of LocalResources based on the blobstore-map passed in."
    +  [blobstore-map]
    +  (if blobstore-map
    +    (for [[k, v] blobstore-map] (LocalResource. k (should-uncompress-blob? v)))
    +    ()))
    +
    +(defn add-blob-references
    +  "For each of the downloaded topologies, adds references to the blobs that the topologies are
    +  using. This is used to reconstruct the cache on restart."
    +  [localizer storm-id conf]
    +  (let [storm-conf (read-supervisor-storm-conf conf storm-id)
    +        blobstore-map (storm-conf TOPOLOGY-BLOBSTORE-MAP)
    +        user (storm-conf TOPOLOGY-SUBMITTER-USER)
    +        topo-name (storm-conf TOPOLOGY-NAME)
    +        localresources (blobstore-map-to-localresources blobstore-map)]
    +    (if blobstore-map
    +      (.addReferences localizer localresources user topo-name))))
    +
    +(defn rm-topo-files
    +  [conf storm-id localizer rm-blob-refs?]
    +  (let [path (supervisor-stormdist-root conf storm-id)]
    +  (try
    +    (if rm-blob-refs?
    +      (remove-blob-references localizer storm-id conf))
    +    (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
    +      (rmr-as-user conf storm-id path)
    +      (rmr (supervisor-stormdist-root conf storm-id)))
    +    (catch Exception e
    +      (log-message e (str "Exception removing: " storm-id))))))
    +
    +(defn verify-downloaded-files [conf localizer assigned-storm-ids all-downloaded-storm-ids]
    +  "Check for the files exists to avoid supervisor crashing
    +   Also makes sure there is no necessity for locking"
    --- End diff --
    
    This is not a docstring: the docstring needs to go before the parameter list, and the param list should be on its own line.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45760378
  
    --- Diff: external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java ---
    @@ -0,0 +1,383 @@
    +/**
    + * 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 org.apache.storm.hdfs.blobstore;
    +
    +import backtype.storm.Config;
    +import backtype.storm.blobstore.AtomicOutputStream;
    +import backtype.storm.blobstore.AtomicOutputStream;
    +import backtype.storm.blobstore.BlobStore;
    +import backtype.storm.blobstore.BlobStoreAclHandler;
    +import backtype.storm.blobstore.BlobStoreFile;
    +import backtype.storm.blobstore.InputStreamWithMeta;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.utils.NimbusClient;
    --- End diff --
    
    @d2r addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43811771
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -984,25 +1094,30 @@
             ))))
     
     (defn cleanup-corrupt-topologies! [nimbus]
    -  (if (is-leader nimbus :throw-exception false)
    -    (let [storm-cluster-state (:storm-cluster-state nimbus)
    -          code-ids (set (code-ids (:conf nimbus)))
    -          active-topologies (set (.active-storms storm-cluster-state))
    -          corrupt-topologies (set/difference active-topologies code-ids)]
    -      (doseq [corrupt corrupt-topologies]
    -        (log-message "Corrupt topology " corrupt " has state on zookeeper but doesn't have a local dir on Nimbus. Cleaning up...")
    -        (.remove-storm! storm-cluster-state corrupt)
    -        )))
    -  (log-message "not a leader, skipping cleanup-corrupt-topologies"))
    -
    -;;setsup code distributor entries for all current topologies for which code is available locally.
    -(defn setup-code-distributor [nimbus]
       (let [storm-cluster-state (:storm-cluster-state nimbus)
    -        locally-available-storm-ids (set (code-ids (:conf nimbus)))
    +        blob-store (:blob-store nimbus)
    +        code-ids (set (code-ids blob-store))
             active-topologies (set (.active-storms storm-cluster-state))
    -        locally-available-active-storm-ids (set/intersection locally-available-storm-ids active-topologies)]
    -    (doseq [storm-id locally-available-active-storm-ids]
    -      (.setup-code-distributor! storm-cluster-state storm-id (:nimbus-host-port-info nimbus)))))
    +        corrupt-topologies (set/difference active-topologies code-ids)]
    +    (doseq [corrupt corrupt-topologies]
    +      (log-message "Corrupt topology " corrupt " has state on zookeeper but doesn't have a local dir on Nimbus. Cleaning up...")
    +      (.remove-storm! storm-cluster-state corrupt)
    +      (if (instance? LocalFsBlobStore blob-store)
    +        (doseq [blob-key (get-key-list-from-id (:conf nimbus) corrupt)]
    +          (.remove-blobstore-key! storm-cluster-state blob-key)))
    +      )))
    +
    +;;setsup blobstore for all current keys
    +(defn setup-blobstore [nimbus]
    +  (let [storm-cluster-state (:storm-cluster-state nimbus)
    +        blob-store (:blob-store nimbus)
    +        local-list-of-keys (set (get-key-list-from-blob-store blob-store))
    --- End diff --
    
    Just wanted to reuse the set functions, moreover I guess I used the list in other couple of places


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45384344
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStore.java ---
    @@ -0,0 +1,446 @@
    +/**
    + * 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 java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Set;
    +import java.util.regex.Pattern;
    +
    +import javax.security.auth.Subject;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import backtype.storm.daemon.Shutdownable;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +
    +/**
    + * Provides a way to store blobs that can be downloaded.
    + * Blobs must be able to be uploaded and listed from Nimbus,
    + * and downloaded from the Supervisors. It is a key value based
    + * store. Key being a string and value being the blob data.
    + *
    + * ACL checking must take place against the provided subject.
    + * If the blob store does not support Security it must validate
    + * that all ACLs set are always WORLD, everything.
    + *
    + * The users can upload their blobs through the blob store command
    + * line. The command line utilty also allows us to update,
    + * delete.
    + *
    + * Modifying the replication factor only works for HdfsBlobStore
    + * as for the LocalFsBlobStore the replication is dependent on
    + * the number of Nimbodes available.
    + */
    +public abstract class BlobStore implements Shutdownable {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStore.class);
    +  private static final Pattern KEY_PATTERN = Pattern.compile("^[\\w \\t\\.:_-]+$");
    +  protected static final String BASE_BLOBS_DIR_NAME = "blobs";
    +
    +  /**
    +   * Allows us to initialize the blob store
    +   * @param conf The storm configuration
    +   * @param baseDir The directory path to store the blobs
    +   * @param nimbusInfo Contains the nimbus host, port and leadership information.
    +   */
    +  public abstract void prepare(Map conf, String baseDir, NimbusInfo nimbusInfo);
    +
    +  /**
    +   * Creates the blob.
    +   * @param key Key for the blob.
    +   * @param meta Metadata which contains the acls information
    +   * @param who Is the subject creating the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyAlreadyExistsException
    +   */
    +  public abstract AtomicOutputStream createBlob(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException;
    +
    +  /**
    +   * Updates the blob data.
    +   * @param key Key for the blob.
    +   * @param who Is the subject having the write privilege for the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract AtomicOutputStream updateBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Gets the current version of metadata for a blob
    +   * to be viewed by the user or downloaded by the supervisor.
    +   * @param key Key for the blob.
    +   * @param who Is the subject having the read privilege for the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract ReadableBlobMeta getBlobMeta(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Sets the metadata with renewed acls for the blob.
    +   * @param key Key for the blob.
    +   * @param meta Metadata which contains the updated
    +   * acls information.
    +   * @param who Is the subject having the write privilege for the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract void setBlobMeta(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Deletes the blob data and metadata.
    +   * @param key Key for the blob.
    +   * @param who Is the subject having write privilege for the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract void deleteBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Gets the InputStream to read the blob details
    +   * @param key Key for the blob.
    +   * @param who Is the subject having the read privilege for the blob.
    +   * @return InputStreamWithMeta has the additional
    +   * file length and version information.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract InputStreamWithMeta getBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Returns an iterator with all the list of
    +   * keys currently available on the blob store.
    +   * @param who Is the subject creating the blob.
    +   * @return Iterator<String>
    +   */
    +  public abstract Iterator<String> listKeys(Subject who);
    +
    +  /**
    +   * Gets the replication factor of the blob.
    +   * @param key Key for the blob.
    +   * @param who Is the subject having the read privilege for the blob.
    +   * @return BlobReplication object containing the
    +   * replication factor for the blob.
    +   * @throws Exception
    +   */
    +  public abstract int getBlobReplication(String key, Subject who) throws Exception;
    +
    +  /**
    +   * Modifies the replication factor of the blob.
    +   * @param key Key for the blob.
    +   * @param replication The replication factor the
    +   * blob has to be set to.
    +   * @param who Is the subject having the update privilege for the blob
    +   * @return BlobReplication object containing the
    +   * updated replication factor for the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   * @throws IOException
    +   */
    +  public abstract int updateBlobReplication(String key, int replication, Subject who) throws AuthorizationException, KeyNotFoundException, IOException;
    +
    +  /**
    +   * Filters keys based on the KeyFilter
    +   * passed as the argument.
    +   * @param filter Filter passed as the
    +   * @param who Might not want to have the subject as it is not doing anything
    +   * @param <R> Type
    +   * @return Set of filtered keys
    +   */
    +  public <R> Set<R> filterAndListKeys(KeyFilter<R> filter, Subject who) {
    +    Set<R> ret = new HashSet<R>();
    +    Iterator<String> keys = listKeys(who);
    +    while (keys.hasNext()) {
    +      String key = keys.next();
    +      R filtered = filter.filter(key);
    +      if (filtered != null) {
    +        ret.add(filtered);
    +      }
    +    }
    +    return ret;
    +  }
    +
    +  /**
    +   * Validates key checking for potentially harmful patterns
    +   * @param key Key for the blob.
    +   * @throws AuthorizationException
    +   */
    +  public static final void validateKey(String key) throws AuthorizationException {
    +    if (key == null || key.isEmpty() || "..".equals(key) || ".".equals(key) || !KEY_PATTERN.matcher(key).matches()) {
    +      LOG.error("'{}' does not appear to be valid {}", key, KEY_PATTERN);
    +      throw new AuthorizationException(key+" does not appear to be a valid blob key");
    +    }
    +  }
    +
    +  /**
    +   * Wrapper called to create the blob which contains
    +   * the byte data
    +   * @param key Key for the blob.
    +   * @param data Byte data that needs to be uploaded.
    +   * @param meta Metadata which contains the acls information
    +   * @param who Is the subject creating the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyAlreadyExistsException
    +   * @throws IOException
    +   */
    +  public void createBlob(String key, byte [] data, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException, IOException {
    +    AtomicOutputStream out = null;
    +    try {
    +      out = createBlob(key, meta, who);
    +      out.write(data);
    +      out.close();
    +      out = null;
    +    } finally {
    +      if (out != null) {
    +        out.cancel();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Wrapper called to create the blob which contains
    +   * the byte data
    +   * @param key Key for the blob.
    +   * @param in InputStream from which the data is read to be
    +   * written as a part of the blob.
    +   * @param meta Metadata which contains the acls information
    +   * @param who Is the subject creating the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyAlreadyExistsException
    +   * @throws IOException
    +   */
    +  public void createBlob(String key, InputStream in, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException, IOException {
    +    AtomicOutputStream out = null;
    +    try {
    +      out = createBlob(key, meta, who);
    +      byte[] buffer = new byte[2048];
    +      int len = 0;
    +      while ((len = in.read(buffer)) > 0) {
    +        out.write(buffer, 0, len);
    +      }
    +      out.close();
    +    } catch (AuthorizationException | IOException | RuntimeException e) {
    +      out.cancel();
    --- End diff --
    
    If we catch `AuthorizationException` from the above call to `createBlob`, then `out` will be null here.  So we need to check that `out` is not null here before calling `cancel`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on the pull request:

    https://github.com/apache/storm/pull/845#issuecomment-153477014
  
    storm.thrift:
    ```
    enum AccessControlType {
      OTHER = 1,
      USER = 2
      //eventually ,GROUP=3
    }
    ```
    
    Would it be better to remove this comment and create a JIRA for adding group
    support instead?  In the JIRA, we could point to this spot in the code as a
    hint.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43785974
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -950,20 +1074,33 @@
                 TOPOLOGY-EVENTLOGGER-EXECUTORS (total-conf TOPOLOGY-EVENTLOGGER-EXECUTORS)
                 TOPOLOGY-MAX-TASK-PARALLELISM (total-conf TOPOLOGY-MAX-TASK-PARALLELISM)})))
     
    +(defn blob-rm [blob-store key storm-cluster-state]
    +  (try
    +    (.deleteBlob blob-store key (get-nimbus-subject))
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.remove-blobstore-key! storm-cluster-state key))
    +    (catch Exception e)))
    +
    +(defn rm-from-blob-store [id blob-store storm-cluster-state]
    --- End diff --
    
    Could we rename these?  blob-rm and rm-from-blob-store don't differentiate each other.  I think it would be better to be something like `blob-rm-key` and `blob-rm-topology-keys`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45425144
  
    --- Diff: external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java ---
    @@ -0,0 +1,383 @@
    +/**
    + * 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 org.apache.storm.hdfs.blobstore;
    +
    +import backtype.storm.Config;
    +import backtype.storm.blobstore.AtomicOutputStream;
    +import backtype.storm.blobstore.AtomicOutputStream;
    --- End diff --
    
    Duplicate import


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43908682
  
    --- Diff: storm-core/src/clj/backtype/storm/cluster.clj ---
    @@ -248,9 +256,9 @@
       [id]
       (str ASSIGNMENTS-SUBTREE "/" id))
     
    -(defn code-distributor-path
    -  [id]
    -  (str CODE-DISTRIBUTOR-SUBTREE "/" id))
    +(defn blobstore-path
    +  [key]
    +  (str BLOBSTORE-SUBTREE "/" key))
    --- End diff --
    
    There is a server side validation for this, im adding the client side validation just to reinforce it


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by unsleepy22 <gi...@git.apache.org>.
Github user unsleepy22 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46020661
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStoreAclHandler.java ---
    @@ -0,0 +1,401 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.AccessControl;
    +import backtype.storm.generated.AccessControlType;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.security.auth.AuthUtils;
    +import backtype.storm.security.auth.IPrincipalToLocal;
    +import backtype.storm.security.auth.NimbusPrincipal;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.security.Principal;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Provides common handling of acls for Blobstores.
    + * Also contains some static utility functions related to Blobstores.
    + */
    +public class BlobStoreAclHandler {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStoreAclHandler.class);
    +  private final IPrincipalToLocal _ptol;
    +
    +  public static final int READ = 0x01;
    +  public static final int WRITE = 0x02;
    +  public static final int ADMIN = 0x04;
    +  public static final List<AccessControl> WORLD_EVERYTHING =
    +      Arrays.asList(new AccessControl(AccessControlType.OTHER, READ | WRITE | ADMIN));
    +  public static final List<AccessControl> DEFAULT = new ArrayList<AccessControl>();
    +  private Set<String> _supervisors;
    +  private Set<String> _admins;
    +
    +  public BlobStoreAclHandler(Map conf) {
    +    _ptol = AuthUtils.GetPrincipalToLocalPlugin(conf);
    +    _supervisors = new HashSet<String>();
    +    _admins = new HashSet<String>();
    +    if (conf.containsKey(Config.NIMBUS_SUPERVISOR_USERS)) {
    +      _supervisors.addAll((List<String>)conf.get(Config.NIMBUS_SUPERVISOR_USERS));
    +    }
    +    if (conf.containsKey(Config.NIMBUS_ADMINS)) {
    +      _admins.addAll((List<String>)conf.get(Config.NIMBUS_ADMINS));
    +    }
    +  }
    +
    +  private static AccessControlType parseACLType(String type) {
    +    if ("other".equalsIgnoreCase(type) || "o".equalsIgnoreCase(type)) {
    +      return AccessControlType.OTHER;
    +    } else if ("user".equalsIgnoreCase(type) || "u".equalsIgnoreCase(type)) {
    +      return AccessControlType.USER;
    +    }
    +    throw new IllegalArgumentException(type+" is not a valid access control type");
    +  }
    +
    +  private static int parseAccess(String access) {
    +    int ret = 0;
    +    for (char c: access.toCharArray()) {
    +      if ('r' == c) {
    +        ret = ret | READ;
    +      } else if ('w' == c) {
    +        ret = ret | WRITE;
    +      } else if ('a' == c) {
    +        ret = ret | ADMIN;
    +      } else if ('-' == c) {
    +        //ignored
    +      } else {
    +        throw new IllegalArgumentException("");
    +      }
    +    }
    +    return ret;
    +  }
    +
    +  public static AccessControl parseAccessControl(String str) {
    +    String[] parts = str.split(":");
    +    String type = "other";
    +    String name = "";
    +    String access = "-";
    +    if (parts.length > 3) {
    +      throw new IllegalArgumentException("Don't know how to parse "+str+" into an ACL value");
    +    } else if (parts.length == 1) {
    +      type = "other";
    +      name = "";
    +      access = parts[0];
    +    } else if (parts.length == 2) {
    +      type = "user";
    +      name = parts[0];
    +      access = parts[1];
    +    } else if (parts.length == 3) {
    +      type = parts[0];
    +      name = parts[1];
    +      access = parts[2];
    +    }
    +    AccessControl ret = new AccessControl();
    +    ret.set_type(parseACLType(type));
    +    ret.set_name(name);
    +    ret.set_access(parseAccess(access));
    +    return ret;
    +  }
    +
    +  private static String accessToString(int access) {
    +    StringBuffer ret = new StringBuffer();
    +    ret.append(((access & READ) > 0) ? "r" : "-");
    +    ret.append(((access & WRITE) > 0) ? "w" : "-");
    +    ret.append(((access & ADMIN) > 0) ? "a" : "-");
    +    return ret.toString();
    +  }
    +
    +  public static String accessControlToString(AccessControl ac) {
    +    StringBuffer ret = new StringBuffer();
    +    switch(ac.get_type()) {
    +      case OTHER:
    +        ret.append("o");
    +        break;
    +      case USER:
    +        ret.append("u");
    +        break;
    +      default:
    +        throw new IllegalArgumentException("Don't know what a type of "+ac.get_type()+" means ");
    +    }
    +    ret.append(":");
    +    if (ac.is_set_name()) {
    +      ret.append(ac.get_name());
    +    }
    +    ret.append(":");
    +    ret.append(accessToString(ac.get_access()));
    +    return ret.toString();
    +  }
    +
    +  public static void validateSettableACLs(String key, List<AccessControl> acls) throws AuthorizationException {
    +    Set<String> aclUsers = new HashSet<>();
    +    List<String> duplicateUsers = new ArrayList<>();
    +    for (AccessControl acl : acls) {
    +      String aclUser = acl.get_name();
    +      if (aclUser != null && !aclUser.isEmpty() && !aclUsers.add(aclUser)) {
    +        LOG.error("'{}' user can't appear more than once in the ACLs", aclUser);
    +        duplicateUsers.add(aclUser);
    +      }
    +    }
    +    if (duplicateUsers.size() > 0) {
    +      String errorMessage  = "user " + Arrays.toString(duplicateUsers.toArray())
    +          + " can't appear more than once in the ACLs for key [" + key +"].";
    +      throw new AuthorizationException(errorMessage);
    +    }
    +  }
    +
    +  private Set<String> constructUserFromPrincipals(Subject who) {
    +    Set<String> user = new HashSet<String>();
    +    if (who != null) {
    +      for (Principal p : who.getPrincipals()) {
    +        user.add(_ptol.toLocal(p));
    +      }
    +    }
    +    return user;
    +  }
    +
    +  private boolean isAdmin(Subject who) {
    +    Set<String> user = constructUserFromPrincipals(who);
    +    for (String u : user) {
    +      if (_admins.contains(u)) {
    +        return true;
    +      }
    +    }
    +    return false;
    +  }
    +
    +  private boolean isReadOperation(int operation) {
    +    if (operation == 1) {
    +      return true;
    +    }
    +    return false;
    +  }
    +
    +  private boolean isSupervisor(Subject who, int operation) {
    +    Set<String> user = constructUserFromPrincipals(who);
    +    if (isReadOperation(operation)) {
    +      for (String u : user) {
    +        if (_supervisors.contains(u)) {
    +          return true;
    +        }
    +      }
    +    }
    +    return false;
    +  }
    +
    +  private boolean isNimbus(Subject who) {
    +    Set<Principal> principals = null;
    +    boolean isNimbusInstance = false;
    +    if (who != null) {
    +      principals = who.getPrincipals();
    +      for (Principal principal : principals) {
    +        if (principal instanceof NimbusPrincipal) {
    +          isNimbusInstance = true;
    +        }
    +      }
    +    }
    +    return isNimbusInstance;
    +  }
    +
    +  public boolean checkForValidUsers(Subject who, int mask) {
    +    if (isNimbus(who) || isAdmin(who) || isSupervisor(who,mask)) {
    --- End diff --
    
    return isNimbus(who) || isAdmin(who) || isSupervisor(who,mask);


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45789625
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/KeySequenceNumber.java ---
    @@ -0,0 +1,227 @@
    +/**
    + * 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.zookeeper.CreateMode;
    +import org.apache.zookeeper.ZooDefs;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.nio.ByteBuffer;
    +import java.util.TreeSet;
    +import java.util.Map;
    +import java.util.List;
    +
    +/**
    + * Class hands over the key sequence number which implies the number of updates made to a blob.
    + * The information regarding the keys and the sequence number which represents the number of updates are
    + * stored within the zookeeper in the following format.
    + * /storm/blobstore/key_name/nimbushostport-sequencenumber
    + * Example:
    + * If there are two nimbodes with nimbus.seeds:leader,non-leader are set,
    + * then the state inside the zookeeper is eventually stored as:
    + * /storm/blobstore/key1/leader:8080-1
    + * /storm/blobstore/key1/non-leader:8080-1
    + * indicates that a new blob with the name key1 has been created on the leader
    + * nimbus and the non-leader nimbus syncs after a call back is triggered by attempting
    + * to download the blob and finally updates its state inside the zookeeper.
    + *
    + * A watch is placed on the /storm/blobstore/key1 and the znodes leader:8080-1 and
    + * non-leader:8080-1 are ephemeral which implies that these nodes exist only until the
    + * connection between the corresponding nimbus and the zookeeper persist. If in case the
    + * nimbus crashes the node disappears under /storm/blobstore/key1.
    + *
    + * The sequence number for the keys are handed over based on the following scenario:
    + * Lets assume there are three nimbodes up and running, one being the leader and the other
    + * being the non-leader.
    + *
    + * 1. Create is straight forward.
    + * Check whether the znode -> /storm/blobstore/key1 has been created or not. It implies
    + * the blob has not been created yet. If not created, it creates it and updates the zookeeper
    + * states under /storm/blobstore/key1 and /storm/blobstoremaxkeysequencenumber/key1.
    + * The znodes it creates on these nodes are /storm/blobstore/key1/leader:8080-1,
    + * /storm/blobstore/key1/non-leader:8080-1 and /storm/blobstoremaxkeysequencenumber/key1/1.
    + * The later holds the global sequence number across all nimbodes more like a static variable
    --- End diff --
    
    Really nice documentation here!
    
    `later` -> `latter`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45760253
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStore.java ---
    @@ -0,0 +1,446 @@
    +/**
    + * 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 java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Set;
    +import java.util.regex.Pattern;
    +
    +import javax.security.auth.Subject;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import backtype.storm.daemon.Shutdownable;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +
    +/**
    + * Provides a way to store blobs that can be downloaded.
    + * Blobs must be able to be uploaded and listed from Nimbus,
    + * and downloaded from the Supervisors. It is a key value based
    + * store. Key being a string and value being the blob data.
    + *
    + * ACL checking must take place against the provided subject.
    + * If the blob store does not support Security it must validate
    + * that all ACLs set are always WORLD, everything.
    + *
    + * The users can upload their blobs through the blob store command
    + * line. The command line utilty also allows us to update,
    + * delete.
    + *
    + * Modifying the replication factor only works for HdfsBlobStore
    + * as for the LocalFsBlobStore the replication is dependent on
    + * the number of Nimbodes available.
    + */
    +public abstract class BlobStore implements Shutdownable {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStore.class);
    +  private static final Pattern KEY_PATTERN = Pattern.compile("^[\\w \\t\\.:_-]+$");
    +  protected static final String BASE_BLOBS_DIR_NAME = "blobs";
    +
    +  /**
    +   * Allows us to initialize the blob store
    +   * @param conf The storm configuration
    +   * @param baseDir The directory path to store the blobs
    +   * @param nimbusInfo Contains the nimbus host, port and leadership information.
    +   */
    +  public abstract void prepare(Map conf, String baseDir, NimbusInfo nimbusInfo);
    +
    +  /**
    +   * Creates the blob.
    +   * @param key Key for the blob.
    +   * @param meta Metadata which contains the acls information
    +   * @param who Is the subject creating the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyAlreadyExistsException
    +   */
    +  public abstract AtomicOutputStream createBlob(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException;
    +
    +  /**
    +   * Updates the blob data.
    +   * @param key Key for the blob.
    +   * @param who Is the subject having the write privilege for the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract AtomicOutputStream updateBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Gets the current version of metadata for a blob
    +   * to be viewed by the user or downloaded by the supervisor.
    +   * @param key Key for the blob.
    +   * @param who Is the subject having the read privilege for the blob.
    +   * @return AtomicOutputStream returns a stream into which the data
    +   * can be written into.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract ReadableBlobMeta getBlobMeta(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Sets the metadata with renewed acls for the blob.
    +   * @param key Key for the blob.
    +   * @param meta Metadata which contains the updated
    +   * acls information.
    +   * @param who Is the subject having the write privilege for the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract void setBlobMeta(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Deletes the blob data and metadata.
    +   * @param key Key for the blob.
    +   * @param who Is the subject having write privilege for the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract void deleteBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Gets the InputStream to read the blob details
    +   * @param key Key for the blob.
    +   * @param who Is the subject having the read privilege for the blob.
    +   * @return InputStreamWithMeta has the additional
    +   * file length and version information.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   */
    +  public abstract InputStreamWithMeta getBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
    +
    +  /**
    +   * Returns an iterator with all the list of
    +   * keys currently available on the blob store.
    +   * @param who Is the subject creating the blob.
    +   * @return Iterator<String>
    +   */
    +  public abstract Iterator<String> listKeys(Subject who);
    +
    +  /**
    +   * Gets the replication factor of the blob.
    +   * @param key Key for the blob.
    +   * @param who Is the subject having the read privilege for the blob.
    +   * @return BlobReplication object containing the
    +   * replication factor for the blob.
    +   * @throws Exception
    +   */
    +  public abstract int getBlobReplication(String key, Subject who) throws Exception;
    +
    +  /**
    +   * Modifies the replication factor of the blob.
    +   * @param key Key for the blob.
    +   * @param replication The replication factor the
    +   * blob has to be set to.
    +   * @param who Is the subject having the update privilege for the blob
    +   * @return BlobReplication object containing the
    +   * updated replication factor for the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyNotFoundException
    +   * @throws IOException
    +   */
    +  public abstract int updateBlobReplication(String key, int replication, Subject who) throws AuthorizationException, KeyNotFoundException, IOException;
    +
    +  /**
    +   * Filters keys based on the KeyFilter
    +   * passed as the argument.
    +   * @param filter Filter passed as the
    +   * @param who Might not want to have the subject as it is not doing anything
    +   * @param <R> Type
    +   * @return Set of filtered keys
    +   */
    +  public <R> Set<R> filterAndListKeys(KeyFilter<R> filter, Subject who) {
    +    Set<R> ret = new HashSet<R>();
    +    Iterator<String> keys = listKeys(who);
    +    while (keys.hasNext()) {
    +      String key = keys.next();
    +      R filtered = filter.filter(key);
    +      if (filtered != null) {
    +        ret.add(filtered);
    +      }
    +    }
    +    return ret;
    +  }
    +
    +  /**
    +   * Validates key checking for potentially harmful patterns
    +   * @param key Key for the blob.
    +   * @throws AuthorizationException
    +   */
    +  public static final void validateKey(String key) throws AuthorizationException {
    +    if (key == null || key.isEmpty() || "..".equals(key) || ".".equals(key) || !KEY_PATTERN.matcher(key).matches()) {
    +      LOG.error("'{}' does not appear to be valid {}", key, KEY_PATTERN);
    +      throw new AuthorizationException(key+" does not appear to be a valid blob key");
    +    }
    +  }
    +
    +  /**
    +   * Wrapper called to create the blob which contains
    +   * the byte data
    +   * @param key Key for the blob.
    +   * @param data Byte data that needs to be uploaded.
    +   * @param meta Metadata which contains the acls information
    +   * @param who Is the subject creating the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyAlreadyExistsException
    +   * @throws IOException
    +   */
    +  public void createBlob(String key, byte [] data, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException, IOException {
    +    AtomicOutputStream out = null;
    +    try {
    +      out = createBlob(key, meta, who);
    +      out.write(data);
    +      out.close();
    +      out = null;
    +    } finally {
    +      if (out != null) {
    +        out.cancel();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Wrapper called to create the blob which contains
    +   * the byte data
    +   * @param key Key for the blob.
    +   * @param in InputStream from which the data is read to be
    +   * written as a part of the blob.
    +   * @param meta Metadata which contains the acls information
    +   * @param who Is the subject creating the blob.
    +   * @throws AuthorizationException
    +   * @throws KeyAlreadyExistsException
    +   * @throws IOException
    +   */
    +  public void createBlob(String key, InputStream in, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException, IOException {
    +    AtomicOutputStream out = null;
    +    try {
    +      out = createBlob(key, meta, who);
    +      byte[] buffer = new byte[2048];
    +      int len = 0;
    +      while ((len = in.read(buffer)) > 0) {
    +        out.write(buffer, 0, len);
    +      }
    +      out.close();
    +    } catch (AuthorizationException | IOException | RuntimeException e) {
    +      out.cancel();
    +    } finally {
    +      in.close();
    +    }
    +  }
    +
    +  /**
    +   * Reads the blob from the blob store
    +   * and writes it into the output stream.
    +   * @param key Key for the blob.
    +   * @param out Output stream
    +   * @param who Is the subject creating
    +   * the blob.
    +   * @throws IOException
    +   * @throws KeyNotFoundException
    +   * @throws AuthorizationException
    +   */
    +  public void readBlobTo(String key, OutputStream out, Subject who) throws IOException, KeyNotFoundException, AuthorizationException {
    +    InputStreamWithMeta in = getBlob(key, who);
    +    if (in == null) {
    +      throw new IOException("Could not find " + key);
    +    }
    +    byte[] buffer = new byte[2048];
    +    int len = 0;
    +    try{
    +      while ((len = in.read(buffer)) > 0) {
    +        out.write(buffer, 0, len);
    +      }
    +    } finally {
    +      in.close();
    +      out.flush();
    +    }
    +  }
    +
    +  /**
    +   * Wrapper around readBlobTo which
    +   * returns a ByteArray output stream.
    +   * @param key  Key for the blob.
    +   * @param who Is the subject creating
    +   * the blob.
    +   * @return ByteArrayOutputStream
    +   * @throws IOException
    +   * @throws KeyNotFoundException
    +   * @throws AuthorizationException
    +   */
    +  public byte[] readBlob(String key, Subject who) throws IOException, KeyNotFoundException, AuthorizationException {
    +    ByteArrayOutputStream out = new ByteArrayOutputStream();
    +    readBlobTo(key, out, who);
    +    return out.toByteArray();
    +  }
    +
    +  /**
    +   * Output stream implemetation used for reading the
    --- End diff --
    
    @d2r addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45102927
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -1206,22 +1328,40 @@
          (.set_reset_log_level_timeout_epoch log-config (coerce/to-long timeout))
          (.unset_reset_log_level_timeout_epoch log-config))))
     
    +(defmethod blob-sync-code :distributed [conf nimbus]
    +  (if (not (is-leader nimbus :throw-exception false))
    +    (let [storm-cluster-state (:storm-cluster-state nimbus)
    +          nimbus-host-port-info (:nimbus-host-port-info nimbus)
    +          blob-store-key-list (into [] (get-key-seq-from-blob-store (:blob-store nimbus)))
    +          zk-key-list (into [] (.blobstore storm-cluster-state (fn [] (blob-sync-code conf nimbus))))]
    +      (log-debug "blob-sync-code " "blob-store-keys " blob-store-key-list "zookeeper-keys " zk-key-list)
    +      (let [sync-blobs (doto
    +                          (SyncBlobs. (:blob-store nimbus) conf)
    +                          (.setNimbusInfo nimbus-host-port-info)
    +                          (.setBlobStoreKeyList (if (not-nil? blob-store-key-list) (ArrayList. blob-store-key-list) (ArrayList.)))
    +                          (.setZookeeperKeyList (if (not-nil? zk-key-list) (ArrayList. zk-key-list) (ArrayList.))))]
    --- End diff --
    
    Do we need to send `ArrayList`s specifically when calling `setBlobStoreKeyList` and `setZookeeperKeyList`, or can we say something like `(or blob-store-key-list [])`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45262050
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -476,14 +539,21 @@
                                                (:sync-retry supervisor))
               new-assignment (->> all-assignment
                                   (filter-key #(.confirmAssigned isupervisor %)))
    -          assigned-storm-ids (assigned-storm-ids-from-port-assignments new-assignment)]
    +          rm-blob-refs? true
    +          assigned-storm-ids (assigned-storm-ids-from-port-assignments new-assignment)
    +          localizer (:localizer supervisor)
    +          checked-downloaded-storm-ids (set (verify-downloaded-files conf localizer assigned-storm-ids all-downloaded-storm-ids))
    +          downloaded-storm-ids (set/difference all-downloaded-storm-ids checked-downloaded-storm-ids)]
    +
           (log-debug "Synchronizing supervisor")
           (log-debug "Storm code map: " storm-code-map)
    -      (log-debug "Downloaded storm ids: " downloaded-storm-ids)
           (log-debug "All assignment: " all-assignment)
           (log-debug "New assignment: " new-assignment)
    +      (log-debug "Assigned Storm Ids" assigned-storm-ids)
    +      (log-debug "All Downloaded Ids" all-downloaded-storm-ids)
    +      (log-debug "Checked Downloaded Ids" checked-downloaded-storm-ids)
    +      (log-debug "Downloaded Ids" downloaded-storm-ids)
    --- End diff --
    
    minor thing: Could we add spaces to clean up the log messages here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on the pull request:

    https://github.com/apache/storm/pull/845#issuecomment-158284600
  
    ConfigValidation.java
    ```
    480    /**
    481     * Validate topology.map config
    482     */
    483     public static class MapOfStringToMapOfStringToObjectValidator extends Validator {
    ```
    It's more useful to describe what the validator does, if necessary, not who might be using it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43934589
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -346,53 +400,104 @@
           [(.getNodeId slot) (.getPort slot)]
           )))
     
    +(defn- get-metadata-version [blob-store key subject]
    +  (let [blob-meta (.getBlobMeta blob-store key subject)]
    +    (.get_version blob-meta)))
    +
    +(defn get-key-set-from-blob-store [blob-store]
    +  (let [key-iter (.listKeys blob-store get-nimbus-subject)]
    +    (set (iterator-seq key-iter))))
    +
     (defn- setup-storm-code [nimbus conf storm-id tmp-jar-location storm-conf topology]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -   (log-message "nimbus file location:" stormroot)
    -   (FileUtils/forceMkdir (File. stormroot))
    -   (FileUtils/cleanDirectory (File. stormroot))
    -   (setup-jar conf tmp-jar-location stormroot)
    -   (FileUtils/writeByteArrayToFile (File. (master-stormcode-path stormroot)) (Utils/serialize topology))
    -   (FileUtils/writeByteArrayToFile (File. (master-stormconf-path stormroot)) (Utils/toCompressedJsonConf storm-conf))
    -   (if (:code-distributor nimbus) (.upload (:code-distributor nimbus) stormroot storm-id))
    -   ))
    +  (let [subject (get-subject)
    +        storm-cluster-state (:storm-cluster-state nimbus)
    +        blob-store (:blob-store nimbus)
    +        jar-key (master-stormjar-key storm-id)
    +        code-key (master-stormcode-key storm-id)
    +        conf-key (master-stormconf-key storm-id)
    +        nimbus-host-port-info (:nimbus-host-port-info nimbus)]
    +    (if tmp-jar-location ;;in local mode there is no jar
    +      (do
    +        (.createBlob blob-store jar-key (FileInputStream. tmp-jar-location) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +        (if (instance? LocalFsBlobStore blob-store)
    +          (.setup-blobstore! storm-cluster-state jar-key nimbus-host-port-info (get-metadata-version blob-store jar-key subject)))))
    +    (.createBlob blob-store conf-key (Utils/toCompressedJsonConf storm-conf) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state conf-key nimbus-host-port-info (get-metadata-version blob-store conf-key subject)))
    +    (.createBlob blob-store code-key (Utils/serialize topology) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state code-key nimbus-host-port-info (get-metadata-version blob-store code-key subject)))))
    +
    +(defn- read-storm-topology [storm-id blob-store]
    +  (Utils/deserialize
    +    (.readBlob blob-store (master-stormcode-key storm-id) (get-subject)) StormTopology))
    +
    +(defn- get-blob-replication-count [blob-key nimbus]
    +  (if (:blob-store nimbus)
    +          (-> (:blob-store nimbus)
    +            (.getBlobReplication  blob-key get-nimbus-subject)
    +            (.get_replication))))
     
     (defn- wait-for-desired-code-replication [nimbus conf storm-id]
       (let [min-replication-count (conf TOPOLOGY-MIN-REPLICATION-COUNT)
             max-replication-wait-time (conf TOPOLOGY-MAX-REPLICATION-WAIT-TIME-SEC)
    -        total-wait-time (atom 0)
    -        current-replication-count (atom (if (:code-distributor nimbus) (.getReplicationCount (:code-distributor nimbus) storm-id) 0))]
    -  (if (:code-distributor nimbus)
    -    (while (and (> min-replication-count @current-replication-count)
    -             (or (= -1 max-replication-wait-time)
    -               (< @total-wait-time max-replication-wait-time)))
    +        current-replication-count-jar (if (not (local-mode? conf)) (atom
    +                                                                     (get-blob-replication-count (master-stormjar-key storm-id) nimbus))
    +                                                                     (atom min-replication-count))
    +        current-replication-count-code (atom (get-blob-replication-count (master-stormcode-key storm-id) nimbus))
    +        current-replication-count-conf (atom (get-blob-replication-count (master-stormconf-key storm-id) nimbus))
    +        total-wait-time (atom 0)]
    +    (log-message "wait for desired replication" "count"
    +      min-replication-count "wait-time" max-replication-wait-time
    +      "code" @current-replication-count-code
    +      "conf" @current-replication-count-conf
    +      "jar" @current-replication-count-jar
    +      "replication count" (get-blob-replication-count (master-stormconf-key storm-id) nimbus))
    +    (if (:blob-store nimbus)
    +      (while (and (> min-replication-count @current-replication-count-jar)
    +               (> min-replication-count @current-replication-count-code)
    +               (> min-replication-count @current-replication-count-conf)
    +               (or (= -1 max-replication-wait-time)
    +                 (< @total-wait-time max-replication-wait-time)))
             (sleep-secs 1)
             (log-debug "waiting for desired replication to be achieved.
               min-replication-count = " min-replication-count  " max-replication-wait-time = " max-replication-wait-time
    -          "current-replication-count = " @current-replication-count " total-wait-time " @total-wait-time)
    +          (if (not (local-mode? conf))"current-replication-count for jar key = " @current-replication-count-jar)
    +          "current-replication-count for code key = " @current-replication-count-code
    +          "current-replication-count for conf key = " @current-replication-count-conf
    +          " total-wait-time " @total-wait-time)
             (swap! total-wait-time inc)
    -        (reset! current-replication-count  (.getReplicationCount (:code-distributor nimbus) storm-id))))
    -  (if (< min-replication-count @current-replication-count)
    -    (log-message "desired replication count "  min-replication-count " achieved,
    -      current-replication-count" @current-replication-count)
    -    (log-message "desired replication count of "  min-replication-count " not achieved but we have hit the max wait time "
    -      max-replication-wait-time " so moving on with replication count = " @current-replication-count)
    -    )))
    -
    -(defn- read-storm-topology [conf storm-id]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -    (Utils/deserialize
    -      (FileUtils/readFileToByteArray
    -        (File. (master-stormcode-path stormroot))
    -        ) StormTopology)))
    +        (if (not (local-mode? conf))(reset! current-replication-count-conf  (get-blob-replication-count (master-stormconf-key storm-id))))
    +          (reset! current-replication-count-code  (get-blob-replication-count (master-stormcode-key storm-id)))
    +          (reset! current-replication-count-jar  (get-blob-replication-count (master-stormjar-key storm-id)))))
    +    (if (and (< min-replication-count @current-replication-count-conf)
    +          (< min-replication-count @current-replication-count-code)
    +          (< min-replication-count @current-replication-count-jar))
    --- End diff --
    
    Arguments to `and` should have matching indentation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45433430
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStore.java ---
    @@ -0,0 +1,306 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.utils.Utils;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.thrift.TBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.ByteArrayOutputStream;
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.FileNotFoundException;
    +import java.io.InputStream;
    +
    +import java.util.*;
    +
    +import static backtype.storm.blobstore.BlobStoreAclHandler.ADMIN;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.READ;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.WRITE;
    +
    +/**
    + * Provides a local file system backed blob store implementation for Nimbus.
    + */
    +public class LocalFsBlobStore extends BlobStore {
    +  public static final Logger LOG = LoggerFactory.getLogger(LocalFsBlobStore.class);
    +  private static final String DATA_PREFIX = "data_";
    +  private static final String META_PREFIX = "meta_";
    +  protected BlobStoreAclHandler _aclHandler;
    +  private final String BLOBSTORE_SUBTREE = "/blobstore/";
    +  private NimbusInfo nimbusInfo;
    +  private FileBlobStoreImpl fbs;
    +  private Map conf;
    +
    +  @Override
    +  public void prepare(Map conf, String overrideBase, NimbusInfo nimbusInfo) {
    +    this.conf = conf;
    +    this.nimbusInfo = nimbusInfo;
    +     if (overrideBase == null) {
    +      overrideBase = (String)conf.get(Config.BLOBSTORE_DIR);
    +      if (overrideBase == null) {
    +        overrideBase = (String) conf.get(Config.STORM_LOCAL_DIR);
    +      }
    +    }
    +    File baseDir = new File(overrideBase, BASE_BLOBS_DIR_NAME);
    +    try {
    +      fbs = new FileBlobStoreImpl(baseDir, conf);
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +    _aclHandler = new BlobStoreAclHandler(conf);
    +  }
    +
    +  @Override
    +  public AtomicOutputStream createBlob(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException {
    +    LOG.debug("Creating Blob for key {}", key);
    +    validateKey(key);
    +    _aclHandler.normalizeSettableBlobMeta(key, meta, who, READ | WRITE | ADMIN);
    +    BlobStoreAclHandler.validateSettableACLs(key, meta.get_acl());
    +    _aclHandler.hasPermissions(meta.get_acl(), READ | WRITE | ADMIN, who, key);
    +    if (fbs.exists(DATA_PREFIX+key)) {
    +      throw new KeyAlreadyExistsException(key);
    +    }
    +    BlobStoreFileOutputStream mOut = null;
    +    try {
    +      mOut = new BlobStoreFileOutputStream(fbs.write(META_PREFIX+key, true));
    +      mOut.write(Utils.thriftSerialize((TBase) meta));
    +      mOut.close();
    +      mOut = null;
    +      return new BlobStoreFileOutputStream(fbs.write(DATA_PREFIX+key, true));
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    } finally {
    +      if (mOut != null) {
    +        try {
    +          mOut.cancel();
    +        } catch (IOException e) {
    +          //Ignored
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public AtomicOutputStream updateBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    validateKey(key);
    +    checkForBlobOrDownload(key);
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(meta.get_acl(), WRITE, who, key);
    +    try {
    +      return new BlobStoreFileOutputStream(fbs.write(DATA_PREFIX+key, false));
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  private SettableBlobMeta getStoredBlobMeta(String key) throws KeyNotFoundException {
    +    InputStream in = null;
    +    try {
    +      LocalFsBlobStoreFile pf = fbs.read(META_PREFIX+key);
    +      try {
    +        in = pf.getInputStream();
    +      } catch (FileNotFoundException fnf) {
    +        throw new KeyNotFoundException(key);
    +      }
    +      ByteArrayOutputStream out = new ByteArrayOutputStream();
    +      byte [] buffer = new byte[2048];
    +      int len;
    +      while ((len = in.read(buffer)) > 0) {
    +        out.write(buffer, 0, len);
    +      }
    +      in.close();
    +      in = null;
    +      SettableBlobMeta sbm = Utils.thriftDeserialize(SettableBlobMeta.class, out.toByteArray());
    +      return sbm;
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    } finally {
    +      if (in != null) {
    +        try {
    +          in.close();
    +        } catch (IOException e) {
    +          //Ignored
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public ReadableBlobMeta getBlobMeta(String key, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    validateKey(key);
    +    if(!checkForBlobOrDownload(key)) {
    +      checkForBlobUpdate(key);
    +    }
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.validateUserCanReadMeta(meta.get_acl(), who, key);
    +    ReadableBlobMeta rbm = new ReadableBlobMeta();
    +    rbm.set_settable(meta);
    +    try {
    +      LocalFsBlobStoreFile pf = fbs.read(DATA_PREFIX+key);
    +      rbm.set_version(pf.getModTime());
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +    return rbm;
    +  }
    +
    +  @Override
    +  public void setBlobMeta(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    validateKey(key);
    +    checkForBlobOrDownload(key);
    +    _aclHandler.normalizeSettableBlobMeta(key, meta, who, ADMIN);
    +    BlobStoreAclHandler.validateSettableACLs(key, meta.get_acl());
    +    SettableBlobMeta orig = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(orig.get_acl(), ADMIN, who, key);
    +    BlobStoreFileOutputStream mOut = null;
    +    try {
    +      mOut = new BlobStoreFileOutputStream(fbs.write(META_PREFIX+key, false));
    +      mOut.write(Utils.thriftSerialize((TBase) meta));
    +      mOut.close();
    +      mOut = null;
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    } finally {
    +      if (mOut != null) {
    +        try {
    +          mOut.cancel();
    +        } catch (IOException e) {
    +          //Ignored
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void deleteBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    validateKey(key);
    +    checkForBlobOrDownload(key);
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(meta.get_acl(), WRITE, who, key);
    +    try {
    +      fbs.deleteKey(DATA_PREFIX+key);
    +      fbs.deleteKey(META_PREFIX+key);
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override
    +  public InputStreamWithMeta getBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    validateKey(key);
    +    if(!checkForBlobOrDownload(key)) {
    +      checkForBlobUpdate(key);
    +    }
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(meta.get_acl(), READ, who, key);
    +    try {
    +      return new BlobStoreFileInputStream(fbs.read(DATA_PREFIX+key));
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override
    +  public Iterator<String> listKeys(Subject who) {
    +    try {
    +      return new KeyTranslationIterator(fbs.listKeys(), DATA_PREFIX);
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override
    +  public void shutdown() {
    +  }
    +
    +  @Override
    +  public int getBlobReplication(String key, Subject who) throws Exception {
    +    validateKey(key);
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(meta.get_acl(), READ, who, key);
    +    CuratorFramework zkClient = Utils.createZKClient(conf);
    +    if (zkClient.checkExists().forPath(BLOBSTORE_SUBTREE + key) == null) {
    +       zkClient.close();
    +       return 0;
    +    }
    +    int replicationCount = zkClient.getChildren().forPath(BLOBSTORE_SUBTREE + key).size();
    +    zkClient.close();
    +    return replicationCount;
    +  }
    +
    +  @Override
    +  public int updateBlobReplication(String key, int replication, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    int replicationCount = 0;
    +    validateKey(key);
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(meta.get_acl(), WRITE, who, key);
    +    try {
    +      LOG.warn ("For local file system blob store the update blobs function does not work." +
    +                "Please use HDFS blob store to make this feature available. The replication your " +
    +                "are noticing is the present replication of the blob based on its availability on various nimbuses");
    +      replicationCount = this.getBlobReplication(key, who);
    +    } catch (Exception e) {
    +      LOG.error("Exception {}", e);
    --- End diff --
    
    Why do we log the exception here and not throw it?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r44998501
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/KeyVersion.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * 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.zookeeper.CreateMode;
    +import org.apache.zookeeper.ZooDefs;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.TreeSet;
    +import java.util.Map;
    +import java.util.List;
    +
    +/**
    + * Class hands over the version of the key to be stored within the zookeeper
    + */
    +public class KeyVersion {
    +  private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
    +  private final String BLOBSTORE_SUBTREE="/blobstore";
    +  private final String BLOBSTORE_KEY_COUNTER_SUBTREE="/blobstorekeycounter";
    +  private String key;
    +  private NimbusInfo nimbusInfo;
    +
    +  public KeyVersion(String key, NimbusInfo nimbusInfo) {
    +    this.key = key;
    +    this.nimbusInfo = nimbusInfo;
    +  }
    +
    +  public int getKey(Map conf) {
    --- End diff --
    
    yes sure, will change it thanks for catching it


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45122806
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -1767,6 +1900,169 @@
                                       topology-id
                                       (doto (GetInfoOptions.) (.set_num_err_choice NumErrorsChoice/ALL))))
     
    +      ;;Blobstore implementation code
    +      (^String beginCreateBlob [this
    +                                ^String blob-key
    +                                ^SettableBlobMeta blob-meta]
    +        (let [session-id (uuid)]
    +          (.put (:blob-uploaders nimbus)
    +            session-id
    +            (->> (ReqContext/context)
    +              (.subject)
    +              (.createBlob (:blob-store nimbus) blob-key blob-meta)))
    +          (log-message "Created blob for " blob-key
    +            " with session id " session-id)
    +          (str session-id)))
    +
    +      (^String beginUpdateBlob [this ^String blob-key]
    +        (let [^AtomicOutputStream os (->> (ReqContext/context)
    +                                       (.subject)
    +                                       (.updateBlob (:blob-store nimbus)
    +                                         blob-key))]
    +          (let [session-id (uuid)]
    +            (.put (:blob-uploaders nimbus) session-id os)
    +            (log-message "Created upload session for " blob-key
    +              " with id " session-id)
    +            (str session-id))))
    +
    +      (^void createStateInZookeeper [this ^String blob-key]
    +        (let [storm-cluster-state (:storm-cluster-state nimbus)
    +              blob-store (:blob-store nimbus)
    +              nimbus-host-port-info (:nimbus-host-port-info nimbus)
    +              conf (:conf nimbus)]
    +          (if (instance? LocalFsBlobStore blob-store)
    +              (.setup-blobstore! storm-cluster-state blob-key nimbus-host-port-info (get-version-for-key blob-key nimbus-host-port-info conf)))
    +          (log-debug "Created state in zookeeper" storm-cluster-state blob-store nimbus-host-port-info)))
    +
    +      (^void uploadBlobChunk [this ^String session ^ByteBuffer blob-chunk]
    +        (let [uploaders (:blob-uploaders nimbus)]
    +          (if-let [^AtomicOutputStream os (.get uploaders session)]
    +            (let [chunk-array (.array blob-chunk)
    +                  remaining (.remaining blob-chunk)
    +                  array-offset (.arrayOffset blob-chunk)
    +                  position (.position blob-chunk)]
    +              (.write os chunk-array (+ array-offset position) remaining)
    +              (.put uploaders session os))
    +            (throw-runtime "Blob for session "
    +              session
    +              " does not exist (or timed out)"))))
    +
    +      (^void finishBlobUpload [this ^String session]
    +        (if-let [^AtomicOutputStream os (.get (:blob-uploaders nimbus) session)]
    +          (do
    +            (.close os)
    +            (log-message "Finished uploading blob for session "
    +              session
    +              ". Closing session.")
    +            (.remove (:blob-uploaders nimbus) session))
    +          (throw-runtime "Blob for session "
    +            session
    +            " does not exist (or timed out)")))
    +
    +      (^void cancelBlobUpload [this ^String session]
    +        (if-let [^AtomicOutputStream os (.get (:blob-uploaders nimbus) session)]
    +          (do
    +            (.cancel os)
    +            (log-message "Canceled uploading blob for session "
    +              session
    +              ". Closing session.")
    +            (.remove (:blob-uploaders nimbus) session))
    +          (throw-runtime "Blob for session "
    +            session
    +            " does not exist (or timed out)")))
    +
    +      (^ReadableBlobMeta getBlobMeta [this ^String blob-key]
    +        (let [^ReadableBlobMeta ret (->> (ReqContext/context)
    +                                      (.subject)
    +                                      (.getBlobMeta (:blob-store nimbus)
    +                                        blob-key))]
    +          ret))
    +
    +      (^void setBlobMeta [this ^String blob-key ^SettableBlobMeta blob-meta]
    +        (->> (ReqContext/context)
    +          (.subject)
    +          (.setBlobMeta (:blob-store nimbus) blob-key blob-meta)))
    +
    +      (^BeginDownloadResult beginBlobDownload [this ^String blob-key]
    +        (let [^InputStreamWithMeta is (->> (ReqContext/context)
    +                                        (.subject)
    +                                        (.getBlob (:blob-store nimbus)
    +                                          blob-key))]
    +          (let [session-id (uuid)
    +                ret (BeginDownloadResult. (.getVersion is) (str session-id))]
    +            (.set_data_size ret (.getFileLength is))
    +            (.put (:blob-downloaders nimbus) session-id (BufferInputStream. is ^Integer (Utils/getInt (conf STORM-BLOBSTORE-INPUTSTREAM-BUFFER-SIZE-BYTES) (int 65536))))
    +            (log-message "Created download session for " blob-key
    +              " with id " session-id)
    +            ret)))
    +
    +      (^ByteBuffer downloadBlobChunk [this ^String session]
    +        (let [downloaders (:blob-downloaders nimbus)
    +              ^BufferInputStream is (.get downloaders session)]
    +          (when-not is
    +            (throw (RuntimeException.
    +                     "Could not find input stream for session " session)))
    +          (let [ret (.read is)]
    +            (.put downloaders session is)
    +            (when (empty? ret)
    +              (.close is)
    +              (.remove downloaders session))
    +            (log-debug "Sending " (alength ret) " bytes")
    +            (ByteBuffer/wrap ret))))
    +
    +      (^void deleteBlob [this ^String blob-key]
    +        (let [subject (->> (ReqContext/context)
    +                           (.subject))]
    +        (.deleteBlob (:blob-store nimbus) blob-key subject)
    +        (when (instance? LocalFsBlobStore blob-store)
    +          (.remove-blobstore-key! (:storm-cluster-state nimbus) blob-key)
    +          (.remove-key-version! (:storm-cluster-state nimbus) blob-key))
    +        (log-message "Deleted blob for key " blob-key)))
    --- End diff --
    
    Check indentation after `let`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by kishorvpatil <gi...@git.apache.org>.
Github user kishorvpatil commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43821508
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/HdfsBlobStoreImpl.java ---
    @@ -0,0 +1,307 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.BlobReplication;
    +import backtype.storm.utils.Utils;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FileStatus;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.fs.permission.FsPermission;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.FileNotFoundException;
    +import java.io.IOException;
    +import java.util.*;
    --- End diff --
    
    explicit import


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45301554
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -732,25 +854,90 @@
      (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
       (worker-launcher-and-wait conf (storm-conf TOPOLOGY-SUBMITTER-USER) ["code-dir" dir] :log-prefix (str "setup conf for " dir))))
     
    +(defn setup-blob-permission [conf storm-conf path]
    +  (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
    +    (worker-launcher-and-wait conf (storm-conf TOPOLOGY-SUBMITTER-USER) ["blob" path] :log-prefix (str "setup blob permissions for " path))))
    +
    +(defn setup-storm-code-dir [conf storm-conf dir]
    +  (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
    +    (worker-launcher-and-wait conf (storm-conf TOPOLOGY-SUBMITTER-USER) ["code-dir" dir] :log-prefix (str "setup conf for " dir))))
    +
    +(defn download-blobs-for-topology!
    +  "Download all blobs listed in the topology configuration for a given topology."
    +  [conf stormconf-path localizer tmproot]
    +  (let [storm-conf (read-supervisor-storm-conf-given-path conf stormconf-path)
    +        blobstore-map (storm-conf TOPOLOGY-BLOBSTORE-MAP)
    +        user (storm-conf TOPOLOGY-SUBMITTER-USER)
    +        topo-name (storm-conf TOPOLOGY-NAME)
    +        user-dir (.getLocalUserFileCacheDir localizer user)
    +        localresources (blobstore-map-to-localresources blobstore-map)]
    +    (when localresources
    +      (when-not (.exists user-dir)
    +        (FileUtils/forceMkdir user-dir)
    +        (setup-blob-permission conf storm-conf (.toString user-dir)))
    +      (try
    +        (let [localized-resources (.getBlobs localizer localresources user topo-name user-dir)]
    +          (setup-blob-permission conf storm-conf (.toString user-dir))
    +          (doseq [local-rsrc localized-resources]
    +            (let [rsrc-file-path (File. (.getFilePath local-rsrc))
    +                  key-name (.getName rsrc-file-path)
    +                  blob-symlink-target-name (.getName (File. (.getCurrentSymlinkPath local-rsrc)))
    +                  symlink-name (get-blob-localname (get blobstore-map key-name) key-name)]
    +              (create-symlink! tmproot (.getParent rsrc-file-path) symlink-name
    +                blob-symlink-target-name))))
    +        (catch AuthorizationException authExp
    +          (log-error authExp))
    +        (catch KeyNotFoundException knf
    +          (log-error knf))))))
    +
    +(defn get-blob-file-names
    +  [blobstore-map]
    +  (if blobstore-map
    +    (for [[k, data] blobstore-map]
    +      (get-blob-localname data k))))
    +
    +(defn download-blobs-for-topology-succeed?
    +  "Assert if all blobs are downloaded for the given topology"
    +  [stormconf-path target-dir]
    +  (let [storm-conf (clojurify-structure (Utils/fromCompressedJsonConf (FileUtils/readFileToByteArray (File. stormconf-path))))
    +        blobstore-map (storm-conf TOPOLOGY-BLOBSTORE-MAP)
    +        file-names (get-blob-file-names blobstore-map)]
    +    (if (and file-names (> (count file-names) 0))
    +      (every? #(Utils/checkFileExists target-dir %) file-names)
    +      true)))
    +
     ;; distributed implementation
     (defmethod download-storm-code
    -    :distributed [conf storm-id master-code-dir supervisor download-lock]
    -    ;; Downloading to permanent location is atomic
    -    (let [tmproot (str (supervisor-tmp-dir conf) file-path-separator (uuid))
    -          stormroot (supervisor-stormdist-root conf storm-id)
    -          master-meta-file-path (master-storm-metafile-path master-code-dir)
    -          supervisor-meta-file-path (supervisor-storm-metafile-path tmproot)]
    -      (locking download-lock
    -        (log-message "Downloading code for storm id " storm-id " from " master-code-dir)
    -        (FileUtils/forceMkdir (File. tmproot))
    -        (Utils/downloadFromMaster conf master-meta-file-path supervisor-meta-file-path)
    -        (if (:code-distributor supervisor)
    -          (.download (:code-distributor supervisor) storm-id (File. supervisor-meta-file-path)))
    -        (extract-dir-from-jar (supervisor-stormjar-path tmproot) RESOURCES-SUBDIR tmproot)
    -        (if (.exists (File. stormroot)) (FileUtils/forceDelete (File. stormroot)))
    -        (FileUtils/moveDirectory (File. tmproot) (File. stormroot))
    -        (setup-storm-code-dir conf (read-supervisor-storm-conf conf storm-id) stormroot)
    -        (log-message "Finished downloading code for storm id " storm-id " from " master-code-dir))))
    +  :distributed [conf storm-id master-code-dir localizer]
    +  ;; Downloading to permanent location is atomic
    +  (let [tmproot (str (supervisor-tmp-dir conf) file-path-separator (uuid))
    +        stormroot (supervisor-stormdist-root conf storm-id)
    +        blobstore (Utils/getSupervisorBlobStore conf)]
    --- End diff --
    
    sure will do that, thanks for you suggestion


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43772986
  
    --- Diff: storm-core/src/clj/backtype/storm/command/blobstore.clj ---
    @@ -0,0 +1,166 @@
    +;; 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.
    +(ns backtype.storm.command.blobstore
    +  (:import [java.io InputStream OutputStream])
    +  (:use [backtype.storm config])
    +  (:import [backtype.storm.generated SettableBlobMeta AccessControl AuthorizationException
    +            KeyNotFoundException])
    +  (:import [backtype.storm.blobstore BlobStoreAclHandler])
    +  (:use [clojure.string :only [split]])
    +  (:use [clojure.tools.cli :only [cli]])
    +  (:use [clojure.java.io :only [copy input-stream output-stream]])
    +  (:use [backtype.storm blobstore log util])
    +  (:gen-class))
    +
    +(defn update-blob-from-stream
    +  "Update a blob in the blob store from an InputStream"
    +  [key ^InputStream in]
    +  (with-configured-blob-client blobstore
    +    (let [out (.updateBlob blobstore key)]
    +      (try 
    +        (copy in out)
    +        (.close out)
    +        (.createStateInZookeeper blobstore key)
    --- End diff --
    
    Shouldn't the blobstore handle this itself when the output is committed/closed?  It seems bad that the client has to ask for the ZK state to be updated separate from interacting with the output stream.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on the pull request:

    https://github.com/apache/storm/pull/845#issuecomment-158982859
  
    > This blob version depends on the modified timestamp of the blob. The versions or sequence numbers I hand over for nimbus HA are different as I have written a KeyVersion class to hand over the version details for synchronizing various blobs on several nimbodes
    
    Ah, OK.  That's clear.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43785957
  
    --- Diff: storm-core/src/clj/backtype/storm/command/blobstore.clj ---
    @@ -0,0 +1,166 @@
    +;; 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.
    +(ns backtype.storm.command.blobstore
    +  (:import [java.io InputStream OutputStream])
    +  (:use [backtype.storm config])
    +  (:import [backtype.storm.generated SettableBlobMeta AccessControl AuthorizationException
    +            KeyNotFoundException])
    +  (:import [backtype.storm.blobstore BlobStoreAclHandler])
    +  (:use [clojure.string :only [split]])
    +  (:use [clojure.tools.cli :only [cli]])
    +  (:use [clojure.java.io :only [copy input-stream output-stream]])
    +  (:use [backtype.storm blobstore log util])
    +  (:gen-class))
    +
    +(defn update-blob-from-stream
    +  "Update a blob in the blob store from an InputStream"
    +  [key ^InputStream in]
    +  (with-configured-blob-client blobstore
    +    (let [out (.updateBlob blobstore key)]
    +      (try 
    +        (copy in out)
    +        (.close out)
    +        (.createStateInZookeeper blobstore key)
    +        (catch Exception e
    +          (log-message e)
    +          (.cancel out)
    +          (throw e))))))
    +
    +(defn create-blob-from-stream
    +  "Create a blob in the blob storm from an InputStream"
    +  [key ^InputStream in ^SettableBlobMeta meta]
    +  (with-configured-blob-client blobstore
    +    (let [out (.createBlob blobstore key meta)]
    +      (try 
    +        (copy in out)
    +        (.close out)
    +        (.createStateInZookeeper blobstore key)
    +        (catch Exception e 
    +          (.cancel out)
    +          (throw e))))))
    +
    +(defn read-blob
    +  "Read a blob in the blob storm and write to an OutputStream"
    --- End diff --
    
    "blob storm"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45124647
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -24,9 +24,14 @@
                [org.apache.commons.io FileUtils]
                [java.io File])
       (:use [backtype.storm config util log timer local-state])
    +  (:import [backtype.storm.generated AuthorizationException KeyNotFoundException WorkerResources])
    +  (:import [java.util.concurrent Executors])
       (:import [backtype.storm.utils VersionInfo])
    +  (:import [java.nio.file Files Path Paths StandardCopyOption])
    +  (:import [backtype.storm.blobstore BlobStoreAclHandler])
    --- End diff --
    
    Unused?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45432871
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStore.java ---
    @@ -0,0 +1,306 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.utils.Utils;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.thrift.TBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.ByteArrayOutputStream;
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.FileNotFoundException;
    +import java.io.InputStream;
    +
    +import java.util.*;
    +
    +import static backtype.storm.blobstore.BlobStoreAclHandler.ADMIN;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.READ;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.WRITE;
    +
    +/**
    + * Provides a local file system backed blob store implementation for Nimbus.
    + */
    +public class LocalFsBlobStore extends BlobStore {
    +  public static final Logger LOG = LoggerFactory.getLogger(LocalFsBlobStore.class);
    +  private static final String DATA_PREFIX = "data_";
    +  private static final String META_PREFIX = "meta_";
    +  protected BlobStoreAclHandler _aclHandler;
    +  private final String BLOBSTORE_SUBTREE = "/blobstore/";
    +  private NimbusInfo nimbusInfo;
    +  private FileBlobStoreImpl fbs;
    +  private Map conf;
    +
    +  @Override
    +  public void prepare(Map conf, String overrideBase, NimbusInfo nimbusInfo) {
    +    this.conf = conf;
    +    this.nimbusInfo = nimbusInfo;
    +     if (overrideBase == null) {
    +      overrideBase = (String)conf.get(Config.BLOBSTORE_DIR);
    +      if (overrideBase == null) {
    +        overrideBase = (String) conf.get(Config.STORM_LOCAL_DIR);
    +      }
    +    }
    +    File baseDir = new File(overrideBase, BASE_BLOBS_DIR_NAME);
    +    try {
    +      fbs = new FileBlobStoreImpl(baseDir, conf);
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +    _aclHandler = new BlobStoreAclHandler(conf);
    +  }
    +
    +  @Override
    +  public AtomicOutputStream createBlob(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException {
    +    LOG.debug("Creating Blob for key {}", key);
    +    validateKey(key);
    +    _aclHandler.normalizeSettableBlobMeta(key, meta, who, READ | WRITE | ADMIN);
    +    BlobStoreAclHandler.validateSettableACLs(key, meta.get_acl());
    +    _aclHandler.hasPermissions(meta.get_acl(), READ | WRITE | ADMIN, who, key);
    +    if (fbs.exists(DATA_PREFIX+key)) {
    +      throw new KeyAlreadyExistsException(key);
    +    }
    +    BlobStoreFileOutputStream mOut = null;
    +    try {
    +      mOut = new BlobStoreFileOutputStream(fbs.write(META_PREFIX+key, true));
    +      mOut.write(Utils.thriftSerialize((TBase) meta));
    +      mOut.close();
    +      mOut = null;
    +      return new BlobStoreFileOutputStream(fbs.write(DATA_PREFIX+key, true));
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    } finally {
    +      if (mOut != null) {
    +        try {
    +          mOut.cancel();
    +        } catch (IOException e) {
    +          //Ignored
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public AtomicOutputStream updateBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    validateKey(key);
    +    checkForBlobOrDownload(key);
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(meta.get_acl(), WRITE, who, key);
    +    try {
    +      return new BlobStoreFileOutputStream(fbs.write(DATA_PREFIX+key, false));
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  private SettableBlobMeta getStoredBlobMeta(String key) throws KeyNotFoundException {
    +    InputStream in = null;
    +    try {
    +      LocalFsBlobStoreFile pf = fbs.read(META_PREFIX+key);
    +      try {
    +        in = pf.getInputStream();
    +      } catch (FileNotFoundException fnf) {
    +        throw new KeyNotFoundException(key);
    +      }
    +      ByteArrayOutputStream out = new ByteArrayOutputStream();
    +      byte [] buffer = new byte[2048];
    +      int len;
    +      while ((len = in.read(buffer)) > 0) {
    +        out.write(buffer, 0, len);
    +      }
    +      in.close();
    +      in = null;
    +      SettableBlobMeta sbm = Utils.thriftDeserialize(SettableBlobMeta.class, out.toByteArray());
    +      return sbm;
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    } finally {
    +      if (in != null) {
    +        try {
    +          in.close();
    +        } catch (IOException e) {
    +          //Ignored
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public ReadableBlobMeta getBlobMeta(String key, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    validateKey(key);
    +    if(!checkForBlobOrDownload(key)) {
    +      checkForBlobUpdate(key);
    +    }
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.validateUserCanReadMeta(meta.get_acl(), who, key);
    +    ReadableBlobMeta rbm = new ReadableBlobMeta();
    +    rbm.set_settable(meta);
    +    try {
    +      LocalFsBlobStoreFile pf = fbs.read(DATA_PREFIX+key);
    +      rbm.set_version(pf.getModTime());
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +    return rbm;
    +  }
    +
    +  @Override
    +  public void setBlobMeta(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    validateKey(key);
    +    checkForBlobOrDownload(key);
    +    _aclHandler.normalizeSettableBlobMeta(key, meta, who, ADMIN);
    +    BlobStoreAclHandler.validateSettableACLs(key, meta.get_acl());
    +    SettableBlobMeta orig = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(orig.get_acl(), ADMIN, who, key);
    +    BlobStoreFileOutputStream mOut = null;
    +    try {
    +      mOut = new BlobStoreFileOutputStream(fbs.write(META_PREFIX+key, false));
    +      mOut.write(Utils.thriftSerialize((TBase) meta));
    --- End diff --
    
    `meta` is already a `TBase`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43780718
  
    --- Diff: storm-core/src/clj/backtype/storm/cluster.clj ---
    @@ -408,13 +416,14 @@
               (swap! assignment-version-callback assoc storm-id callback))
             (get-version cluster-state (assignment-path storm-id) (not-nil? callback)))
     
    -      (code-distributor
    +      ;; blobstore state
    +      (blobstore
             [this callback]
    -        (when callback
    -          (reset! code-distributor-callback callback))
    +        (when callback)
    --- End diff --
    
    This looks broken.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45427569
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/KeyVersion.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * 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.zookeeper.CreateMode;
    +import org.apache.zookeeper.ZooDefs;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.TreeSet;
    +import java.util.Map;
    +import java.util.List;
    +
    +/**
    + * Class hands over the version of the key to be stored within the zookeeper
    + */
    +public class KeyVersion {
    +  private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
    +  private final String BLOBSTORE_SUBTREE="/blobstore";
    +  private final String BLOBSTORE_KEY_COUNTER_SUBTREE="/blobstorekeycounter";
    +  private String key;
    +  private NimbusInfo nimbusInfo;
    +
    +  public KeyVersion(String key, NimbusInfo nimbusInfo) {
    +    this.key = key;
    +    this.nimbusInfo = nimbusInfo;
    +  }
    +
    +  public int getKeyVersion(Map conf) {
    +    TreeSet<Integer> versions = new TreeSet<Integer>();
    +    CuratorFramework zkClient = Utils.createZKClient(conf);
    +    try {
    +      // Key has not been created yet and it is the first time it is being created
    +      if(zkClient.checkExists().forPath(BLOBSTORE_SUBTREE + "/" + key) == null) {
    +        zkClient.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT)
    +                .withACL(ZooDefs.Ids.OPEN_ACL_UNSAFE).forPath(BLOBSTORE_KEY_COUNTER_SUBTREE + "/" + key + "/" + 1);
    --- End diff --
    
    `+ "/1");`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43812125
  
    --- Diff: storm-core/src/clj/backtype/storm/cluster.clj ---
    @@ -408,13 +416,14 @@
               (swap! assignment-version-callback assoc storm-id callback))
             (get-version cluster-state (assignment-path storm-id) (not-nil? callback)))
     
    -      (code-distributor
    +      ;; blobstore state
    +      (blobstore
             [this callback]
             (when callback
    -          (reset! code-distributor-callback callback))
    +          (reset! blobstore-callback callback))
             (do
    -          (sync-path cluster-state CODE-DISTRIBUTOR-SUBTREE)
    -          (get-children cluster-state CODE-DISTRIBUTOR-SUBTREE (not-nil? callback))))
    +          (sync-path cluster-state BLOBSTORE-SUBTREE)
    +          (get-children cluster-state BLOBSTORE-SUBTREE (not-nil? callback))))
    --- End diff --
    
    Seems we can remove the `do` here, since it is implicit.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45132777
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -372,66 +389,50 @@
              ". State: " state
              ", Heartbeat: " (pr-str heartbeat))
             (shutdown-worker supervisor id)
    -        (if (:code-distributor supervisor)
    -          (.cleanup (:code-distributor supervisor) id))
    -        ))
    -
    -    (doseq [id (vals new-worker-ids)]
    -      (local-mkdirs (worker-pids-root conf id))
    -      (local-mkdirs (worker-heartbeats-root conf id)))
    -    (ls-approved-workers! local-state
    -          (merge
    -           (select-keys (ls-approved-workers local-state)
    -                        (keys keepers))
    -           (zipmap (vals new-worker-ids) (keys new-worker-ids))
    -           ))
    -
    -    ;; check storm topology code dir exists before launching workers
    -    (doseq [[port assignment] reassign-executors]
    -      (let [downloaded-storm-ids (set (read-downloaded-storm-ids conf))
    -            storm-id (:storm-id assignment)
    -            cached-assignment-info @(:assignment-versions supervisor)
    -            assignment-info (if (and (not-nil? cached-assignment-info) (contains? cached-assignment-info storm-id ))
    -                              (get cached-assignment-info storm-id)
    -                              (.assignment-info-with-version storm-cluster-state storm-id nil))
    -	    storm-code-map (read-storm-code-locations assignment-info)
    -            master-code-dir (if (contains? storm-code-map :data) (storm-code-map :data))
    -            stormroot (supervisor-stormdist-root conf storm-id)]
    -        (if-not (or (contains? downloaded-storm-ids storm-id) (.exists (File. stormroot)) (nil? master-code-dir))
    -          (download-storm-code conf storm-id master-code-dir supervisor download-lock))
             ))
    -
    -    (wait-for-workers-launch
    -     conf
    -     (dofor [[port assignment] reassign-executors]
    -            (let [id (new-worker-ids port)
    -                  storm-id (:storm-id assignment)
    -                  ^WorkerResources resources (:resources assignment)
    -                  mem-onheap (.get_mem_on_heap resources)]
    -              (try
    -                (log-message "Launching worker with assignment "
    -                             (pr-str assignment)
    -                             " for this supervisor "
    -                             (:supervisor-id supervisor)
    -                             " on port "
    -                             port
    -                             " with id "
    -                             id
    -                             )
    -                (launch-worker supervisor
    -                               (:storm-id assignment)
    -                               port
    -                               id
    -                               mem-onheap)
    -                (mark! supervisor:num-workers-launched)
    -                (catch java.io.FileNotFoundException e
    -                  (log-message "Unable to launch worker due to "
    -                               (.getMessage e)))
    -                (catch java.io.IOException e
    -                  (log-message "Unable to launch worker due to "
    -                               (.getMessage e))))
    -         id)))
    -    ))
    +    (let [valid-new-worker-ids
    +          (into {}
    +            (remove nil?
    +              (dofor [[port assignment] reassign-executors]
    --- End diff --
    
    It might be cleaner here to use `->>` macro.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45397793
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStoreAclHandler.java ---
    @@ -0,0 +1,387 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.AccessControl;
    +import backtype.storm.generated.AccessControlType;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.security.auth.AuthUtils;
    +import backtype.storm.security.auth.IPrincipalToLocal;
    +import backtype.storm.security.auth.NimbusPrincipal;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.security.Principal;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Provides common handling of acls for Blobstores.
    + * Also contains some static utility functions related to Blobstores.
    + */
    +public class BlobStoreAclHandler {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStoreAclHandler.class);
    +  private final IPrincipalToLocal _ptol;
    +
    +  public static final int READ = 0x01;
    +  public static final int WRITE = 0x02;
    +  public static final int ADMIN = 0x04;
    +  public static final List<AccessControl> WORLD_EVERYTHING =
    +      Arrays.asList(new AccessControl(AccessControlType.OTHER, READ | WRITE | ADMIN));
    +  public static final List<AccessControl> DEFAULT = new ArrayList<AccessControl>();
    +  private Set<String> _supervisors;
    +  private Set<String> _admins;
    +
    +  public BlobStoreAclHandler(Map conf) {
    +    _ptol = AuthUtils.GetPrincipalToLocalPlugin(conf);
    +    _supervisors = new HashSet<String>();
    +    _admins = new HashSet<String>();
    +    if (conf.containsKey(Config.NIMBUS_SUPERVISOR_USERS)) {
    +      _supervisors.addAll((List<String>)conf.get(Config.NIMBUS_SUPERVISOR_USERS));
    +    }
    +    if (conf.containsKey(Config.NIMBUS_ADMINS)) {
    +      _admins.addAll((List<String>)conf.get(Config.NIMBUS_ADMINS));
    +    }
    +  }
    +
    +  private static AccessControlType parseACLType(String type) {
    +    if ("other".equalsIgnoreCase(type) || "o".equalsIgnoreCase(type)) {
    +      return AccessControlType.OTHER;
    +    } else if ("user".equalsIgnoreCase(type) || "u".equalsIgnoreCase(type)) {
    +      return AccessControlType.USER;
    +    }
    +    throw new IllegalArgumentException(type+" is not a valid access control type");
    +  }
    +
    +  private static int parseAccess(String access) {
    +    int ret = 0;
    +    for (char c: access.toCharArray()) {
    +      if ('r' == c) {
    +        ret = ret | READ;
    +      } else if ('w' == c) {
    +        ret = ret | WRITE;
    +      } else if ('a' == c) {
    +        ret = ret | ADMIN;
    +      } else if ('-' == c) {
    +        //ignored
    +      } else {
    +        throw new IllegalArgumentException("");
    +      }
    +    }
    +    return ret;
    +  }
    +
    +  public static AccessControl parseAccessControl(String str) {
    +    String[] parts = str.split(":");
    +    String type = "other";
    +    String name = "";
    +    String access = "-";
    +    if (parts.length > 3) {
    +      throw new IllegalArgumentException("Don't know how to parse "+str+" into an ACL value");
    +    } else if (parts.length == 1) {
    +      type = "other";
    +      name = "";
    +      access = parts[0];
    +    } else if (parts.length == 2) {
    +      type = "user";
    +      name = parts[0];
    +      access = parts[1];
    +    } else if (parts.length == 3) {
    +      type = parts[0];
    +      name = parts[1];
    +      access = parts[2];
    +    }
    +    AccessControl ret = new AccessControl();
    +    ret.set_type(parseACLType(type));
    +    ret.set_name(name);
    +    ret.set_access(parseAccess(access));
    +    return ret;
    +  }
    +
    +  private static String accessToString(int access) {
    +    StringBuffer ret = new StringBuffer();
    +    ret.append(((access & READ) > 0) ? "r" : "-");
    +    ret.append(((access & WRITE) > 0) ? "w" : "-");
    +    ret.append(((access & ADMIN) > 0) ? "a" : "-");
    +    return ret.toString();
    +  }
    +
    +  public static String accessControlToString(AccessControl ac) {
    +    StringBuffer ret = new StringBuffer();
    +    switch(ac.get_type()) {
    +      case OTHER:
    +        ret.append("o");
    +        break;
    +      case USER:
    +        ret.append("u");
    +        break;
    +      default:
    +        throw new IllegalArgumentException("Ahh don't know what a type of "+ac.get_type()+" means ");
    +    }
    +    ret.append(":");
    +    if (ac.is_set_name()) {
    +      ret.append(ac.get_name());
    +    }
    +    ret.append(":");
    +    ret.append(accessToString(ac.get_access()));
    +    return ret.toString();
    +  }
    +
    +  public static void validateSettableACLs(String key, List<AccessControl> acls) throws AuthorizationException {
    +    Set<String> aclUsers = new HashSet<>();
    +    List<String> duplicateUsers = new ArrayList<>();
    +    for (AccessControl acl : acls) {
    +      String aclUser = acl.get_name();
    +      if (aclUser != null && !aclUser.isEmpty() && !aclUsers.add(aclUser)) {
    +        LOG.error("'{}' user can't appear more than once in the ACLs", aclUser);
    +        duplicateUsers.add(aclUser);
    +      }
    +    }
    +    if (duplicateUsers.size() > 0) {
    +      String errorMessage  = "user " + Arrays.toString(duplicateUsers.toArray())
    +          + " can't appear more than once in the ACLs for key [" + key +"].";
    +      throw new AuthorizationException(errorMessage);
    +    }
    +  }
    +
    +  private Set<String> constructUserFromPrincipals(Subject who) {
    +    Set<String> user = new HashSet<String>();
    +    if (who == null) {
    +      LOG.debug("in validate acl who is null");
    +    } else {
    +      LOG.debug("in validate acl: " + who);
    +    }
    +    if (who != null) {
    +      for (Principal p : who.getPrincipals()) {
    +        user.add(_ptol.toLocal(p));
    +      }
    +    }
    +    return user;
    +  }
    +
    +  private boolean isSupervisorOrAdmin(Set<String> user, int mask) {
    +    boolean isSupervisor = false;
    +    boolean isAdmin = false;
    +    for(String u : user) {
    +      if (_supervisors.contains(u)) {
    +        isSupervisor = true;
    +        break;
    +      }
    +      if (_admins.contains(u)) {
    +        isAdmin = true;
    +        break;
    +      }
    +    }
    +    if (mask > 0 && !isAdmin) {
    +      isSupervisor = (isSupervisor && (mask == 1));
    +    }
    +    return isSupervisor || isAdmin;
    +  }
    +
    +  private boolean isNimbus(Subject who) {
    +    Set<Principal> principals = null;
    +    boolean isNimbusInstance = false;
    +    if(who != null) {
    +      principals = who.getPrincipals();
    +      for (Principal principal : principals) {
    +        if (principal instanceof NimbusPrincipal) {
    +          isNimbusInstance = true;
    +        }
    +      }
    +    }
    +    return isNimbusInstance;
    +  }
    +
    +  /**
    +   * The user should be able to see the metadata if and only if they have any of READ, WRITE, or ADMIN
    +   */
    +  public void validateUserCanReadMeta(List<AccessControl> acl, Subject who, String key)
    +      throws AuthorizationException {
    +    Set<String> user = constructUserFromPrincipals(who);
    +    if (isNimbus(who)) {
    +      return;
    +    }
    +    if (isSupervisorOrAdmin(user, -1)) {
    +      return;
    +    }
    +    for (AccessControl ac : acl) {
    +      int allowed = getAllowed(ac, user);
    +      LOG.debug(" user: {} allowed: {} key: {}", user, allowed, key);
    +      if ((allowed & (READ | WRITE | ADMIN)) > 0) {
    +        return;
    +      }
    +    }
    +    throw new AuthorizationException(
    +            user + " does not have access to " + key);
    +  }
    --- End diff --
    
    This is mostly duplicate code.  We could just call `validateAnyACL` with the RWA mask.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45630920
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/FileBlobStoreImpl.java ---
    @@ -0,0 +1,248 @@
    +/**
    + * 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.Config;
    +import backtype.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.FileNotFoundException;
    +import java.io.IOException;
    +import java.nio.file.Files;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Timer;
    +import java.util.TimerTask;
    +
    +/**
    + * Very basic blob store impl with no ACL handling.
    + */
    +public class FileBlobStoreImpl {
    +  private static final long FULL_CLEANUP_FREQ = 60 * 60 * 1000l;
    +  private static final int BUCKETS = 1024;
    +  private static final Logger LOG = LoggerFactory.getLogger(FileBlobStoreImpl.class);
    +  private static final Timer timer = new Timer("FileBlobStore cleanup thread", true);
    +
    +  public class KeyInHashDirIterator implements Iterator<String> {
    +    private int currentBucket = 0;
    +    private Iterator<String> it = null;
    +    private String next = null;
    +
    +    public KeyInHashDirIterator() throws IOException {
    +      primeNext();
    +    }
    +
    +    private void primeNext() throws IOException {
    +      while (it == null && currentBucket < BUCKETS) {
    +        String name = String.valueOf(currentBucket);
    +        File dir = new File(_fullPath, name);
    +        try {
    +          it = listKeys(dir);
    +        } catch (FileNotFoundException e) {
    +          it = null;
    +        }
    +        if (it == null || !it.hasNext()) {
    +          it = null;
    +          currentBucket++;
    +        } else {
    +          next = it.next();
    +        }
    +      }
    +    }
    +
    +    @Override
    +    public boolean hasNext() {
    +      return next != null;
    +    }
    +
    +    @Override
    +    public String next() {
    +      if (!hasNext()) {
    +        throw new NoSuchElementException();
    +      }
    +      String current = next;
    +      next = null;
    +      if (it != null) {
    +        if (!it.hasNext()) {
    +          it = null;
    +          currentBucket++;
    +          try {
    +            primeNext();
    +          } catch (IOException e) {
    +            throw new RuntimeException(e);
    +          }
    +        } else {
    +          next = it.next();
    +        }
    +      }
    +      return current;
    +    }
    +
    +    @Override
    +    public void remove() {
    +      throw new UnsupportedOperationException("Delete Not Supported");
    +    }
    +  }
    +
    +  private File _fullPath;
    +  private TimerTask cleanup = null;
    +
    +  public FileBlobStoreImpl(File path, Map<String, Object> conf) throws IOException {
    +    LOG.info("Creating new blob store based in {}", path);
    +    _fullPath = path;
    +    _fullPath.mkdirs();
    +    Object shouldCleanup = conf.get(Config.BLOBSTORE_CLEANUP_ENABLE);
    --- End diff --
    
    `Config.BLOBSTORE_CLEANUP_ENABLE` does not appear to be defined.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43813258
  
    --- Diff: conf/defaults.yaml ---
    @@ -63,7 +63,7 @@ nimbus.supervisor.timeout.secs: 60
     nimbus.monitor.freq.secs: 10
     nimbus.cleanup.inbox.freq.secs: 600
     nimbus.inbox.jar.expiration.secs: 3600
    -nimbus.code.sync.freq.secs: 300
    +nimbus.code.sync.freq.secs: 120
    --- End diff --
    
    ok.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45269398
  
    --- Diff: storm-core/src/jvm/backtype/storm/Config.java ---
    @@ -980,6 +980,126 @@
         public static final String SUPERVISOR_SLOTS_PORTS = "supervisor.slots.ports";
     
         /**
    +     * What blobstore implementation the supervisor should use.
    +     */
    +    @isString
    +    public static final String SUPERVISOR_BLOBSTORE = "supervisor.blobstore.class";
    +    public static final Object SUPERVISOR_BLOBSTORE_SCHEMA = String.class;
    +
    +    /**
    +     * 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.
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String SUPERVISOR_LOCALIZER_CACHE_TARGET_SIZE_MB = "supervisor.localizer.cache.target.size.mb";
    +
    +    /**
    +     * The distributed cache cleanup interval. Controls how often it scans to attempt to cleanup
    +     * anything over the cache target size.
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String SUPERVISOR_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS = "supervisor.localizer.cleanup.interval.ms";
    +
    +    /**
    +     * What blobstore implementation the storm client should use.
    +     */
    +    @isString
    +    public static final String CLIENT_BLOBSTORE = "client.blobstore.class";
    +
    +    /**
    +     * What blobstore download parallelism the supervisor should use.
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String SUPERVISOR_BLOBSTORE_DOWNLOAD_THREAD_COUNT = "supervisor.blobstore.download.thread.count";
    +
    +    /**
    +     * What blobstore download parallelism the supervisor should use.
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String SUPERVISOR_BLOBSTORE_DOWNLOAD_MAX_RETRIES = "supervisor.blobstore.download.max_retries";
    +
    +    /**
    +     * The blobstore super user has all read/write/admin permissions to all blobs - user running
    +     * the blobstore.
    +     */
    +    @isString
    +    public static final String BLOBSTORE_SUPERUSER = "blobstore.superuser";
    +
    +    /**
    +     * What directory to use for the blobstore. The directory is expected to be an
    +     * absolute path when using HDFS blobstore, for LocalFsBlobStore it could be either
    +     * absolute or relative.
    +     */
    +    @isString
    +    public static final String BLOBSTORE_DIR = "blobstore.dir";
    +
    +    /**
    +     * What buffer size to use for the blobstore uploads.
    +     */
    +    @isPositiveNumber
    +    @isInteger
    +    public static final String STORM_BLOBSTORE_INPUTSTREAM_BUFFER_SIZE_BYTES = "storm.blobstore.inputstream.buffer.size.bytes";
    --- End diff --
    
    Is this config useful, or should we pick a hard-coded default?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43799321
  
    --- Diff: pom.xml ---
    @@ -646,11 +650,92 @@
                     <version>${thrift.version}</version>
                     <scope>compile</scope>
                 </dependency>
    +			<!-- used by examples/storm-starter -->
                 <dependency>
                     <groupId>junit</groupId>
    -                <artifactId>junit</artifactId>
    +		        <artifactId>junit</artifactId>
                     <version>${junit.version}</version>
    +		        <scope>test</scope>
    +            </dependency>
    +            <dependency>
    +                <groupId>org.apache.hadoop</groupId>
    +                <artifactId>hadoop-client</artifactId>
    +                <version>${hadoop.version}</version>
    +                <exclusions>
    +                    <exclusion>
    +                        <groupId>log4j</groupId>
    +                        <artifactId>log4j</artifactId>
    +                    </exclusion>
    +                    <exclusion>
    +                        <groupId>org.slf4j</groupId>
    +                        <artifactId>slf4j-api</artifactId>
    +                    </exclusion>
    +                    <exclusion>
    +                        <groupId>org.slf4j</groupId>
    +                        <artifactId>slf4j-log4j12</artifactId>
    +                    </exclusion>
    +                </exclusions>
    +            </dependency>
    --- End diff --
    
    Concerned about dependency conflicts.  Can we have the HDFS-specific deps pulled in from external HDFS instead?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r44986991
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -984,25 +1108,30 @@
             ))))
     
     (defn cleanup-corrupt-topologies! [nimbus]
    -  (if (is-leader nimbus :throw-exception false)
    -    (let [storm-cluster-state (:storm-cluster-state nimbus)
    -          code-ids (set (code-ids (:conf nimbus)))
    -          active-topologies (set (.active-storms storm-cluster-state))
    -          corrupt-topologies (set/difference active-topologies code-ids)]
    -      (doseq [corrupt corrupt-topologies]
    -        (log-message "Corrupt topology " corrupt " has state on zookeeper but doesn't have a local dir on Nimbus. Cleaning up...")
    -        (.remove-storm! storm-cluster-state corrupt)
    -        )))
    -  (log-message "not a leader, skipping cleanup-corrupt-topologies"))
    -
    -;;setsup code distributor entries for all current topologies for which code is available locally.
    -(defn setup-code-distributor [nimbus]
       (let [storm-cluster-state (:storm-cluster-state nimbus)
    -        locally-available-storm-ids (set (code-ids (:conf nimbus)))
    +        blob-store (:blob-store nimbus)
    +        code-ids (set (code-ids blob-store))
             active-topologies (set (.active-storms storm-cluster-state))
    -        locally-available-active-storm-ids (set/intersection locally-available-storm-ids active-topologies)]
    -    (doseq [storm-id locally-available-active-storm-ids]
    -      (.setup-code-distributor! storm-cluster-state storm-id (:nimbus-host-port-info nimbus)))))
    +        corrupt-topologies (set/difference active-topologies code-ids)]
    +    (doseq [corrupt corrupt-topologies]
    +      (log-message "Corrupt topology " corrupt " has state on zookeeper but doesn't have a local dir on Nimbus. Cleaning up...")
    +      (.remove-storm! storm-cluster-state corrupt)
    +      (if (instance? LocalFsBlobStore blob-store)
    +        (doseq [blob-key (get-key-list-from-id (:conf nimbus) corrupt)]
    +          (.remove-blobstore-key! storm-cluster-state blob-key)))
    +      )))
    +
    +;;setsup blobstore for all current keys
    --- End diff --
    
    github collapsed my comment, but the comment was not addressed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by kishorvpatil <gi...@git.apache.org>.
Github user kishorvpatil commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43813189
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -1179,13 +1312,15 @@
                             (fn []
                               (clean-inbox (inbox nimbus) (conf NIMBUS-INBOX-JAR-EXPIRATION-SECS))
                               ))
    -    ;;schedule nimbus code sync thread to sync code from other nimbuses.
    -    (schedule-recurring (:timer nimbus)
    -      0
    -      (conf NIMBUS-CODE-SYNC-FREQ-SECS)
    -      (fn []
    -        (sync-code conf nimbus)
    -        ))
    +
    +    ;; Schedule nimbus code sync thread to sync code from other nimbuses.
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (schedule-recurring (:timer nimbus)
    +                          0
    +                          (conf NIMBUS-CODE-SYNC-FREQ-SECS)
    +                          (fn []
    +                            (blob-sync-code conf nimbus)
    +                            )))
    --- End diff --
    
    format..


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r44985482
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -390,53 +444,99 @@
           [(.getNodeId slot) (.getPort slot)]
           )))
     
    +(defn- get-version-for-key [key nimbus-host-port-info conf]
    +  (let [version (KeyVersion. key nimbus-host-port-info)]
    +    (.getKey version conf)))
    +
    +(defn get-key-seq-from-blob-store [blob-store]
    +  (let [key-iter (.listKeys blob-store nimbus-subject)]
    +    (iterator-seq key-iter)))
    +
     (defn- setup-storm-code [nimbus conf storm-id tmp-jar-location storm-conf topology]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -   (log-message "nimbus file location:" stormroot)
    -   (FileUtils/forceMkdir (File. stormroot))
    -   (FileUtils/cleanDirectory (File. stormroot))
    -   (setup-jar conf tmp-jar-location stormroot)
    -   (FileUtils/writeByteArrayToFile (File. (master-stormcode-path stormroot)) (Utils/serialize topology))
    -   (FileUtils/writeByteArrayToFile (File. (master-stormconf-path stormroot)) (Utils/toCompressedJsonConf storm-conf))
    -   (if (:code-distributor nimbus) (.upload (:code-distributor nimbus) stormroot storm-id))
    -   ))
    +  (let [subject (get-subject)
    +        storm-cluster-state (:storm-cluster-state nimbus)
    +        blob-store (:blob-store nimbus)
    +        jar-key (master-stormjar-key storm-id)
    +        code-key (master-stormcode-key storm-id)
    +        conf-key (master-stormconf-key storm-id)
    +        nimbus-host-port-info (:nimbus-host-port-info nimbus)]
    +    (if tmp-jar-location ;;in local mode there is no jar
    +      (do
    +        (.createBlob blob-store jar-key (FileInputStream. tmp-jar-location) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +        (if (instance? LocalFsBlobStore blob-store)
    +          (.setup-blobstore! storm-cluster-state jar-key nimbus-host-port-info (get-version-for-key jar-key nimbus-host-port-info conf)))))
    --- End diff --
    
    Can use `when` instead of `if ... do` here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45102787
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -15,14 +15,23 @@
     ;; 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.nio ByteBuffer]
                [java.util Collections List HashMap]
                [backtype.storm.generated NimbusSummary])
    -  (:import [java.io FileNotFoundException File FileOutputStream])
    +  (:import [java.util Iterator])
    +  (:import [java.nio ByteBuffer]
    +           [java.util Collections List HashMap ArrayList])
    --- End diff --
    
    We are importing more than once from `java.util`, `backtype.storm.generated`, `backtype.storm.blobstore`, and possibly some others.  We could consolidate these.  I read that the proper way to do this now is to have a single `:import` per file.  Maybe that would clean things up here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45793730
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/KeySequenceNumber.java ---
    @@ -0,0 +1,227 @@
    +/**
    + * 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.zookeeper.CreateMode;
    +import org.apache.zookeeper.ZooDefs;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.nio.ByteBuffer;
    +import java.util.TreeSet;
    +import java.util.Map;
    +import java.util.List;
    +
    +/**
    + * Class hands over the key sequence number which implies the number of updates made to a blob.
    + * The information regarding the keys and the sequence number which represents the number of updates are
    + * stored within the zookeeper in the following format.
    + * /storm/blobstore/key_name/nimbushostport-sequencenumber
    + * Example:
    + * If there are two nimbodes with nimbus.seeds:leader,non-leader are set,
    + * then the state inside the zookeeper is eventually stored as:
    + * /storm/blobstore/key1/leader:8080-1
    + * /storm/blobstore/key1/non-leader:8080-1
    + * indicates that a new blob with the name key1 has been created on the leader
    + * nimbus and the non-leader nimbus syncs after a call back is triggered by attempting
    + * to download the blob and finally updates its state inside the zookeeper.
    + *
    + * A watch is placed on the /storm/blobstore/key1 and the znodes leader:8080-1 and
    + * non-leader:8080-1 are ephemeral which implies that these nodes exist only until the
    + * connection between the corresponding nimbus and the zookeeper persist. If in case the
    + * nimbus crashes the node disappears under /storm/blobstore/key1.
    + *
    + * The sequence number for the keys are handed over based on the following scenario:
    + * Lets assume there are three nimbodes up and running, one being the leader and the other
    + * being the non-leader.
    + *
    + * 1. Create is straight forward.
    + * Check whether the znode -> /storm/blobstore/key1 has been created or not. It implies
    + * the blob has not been created yet. If not created, it creates it and updates the zookeeper
    + * states under /storm/blobstore/key1 and /storm/blobstoremaxkeysequencenumber/key1.
    + * The znodes it creates on these nodes are /storm/blobstore/key1/leader:8080-1,
    + * /storm/blobstore/key1/non-leader:8080-1 and /storm/blobstoremaxkeysequencenumber/key1/1.
    + * The later holds the global sequence number across all nimbodes more like a static variable
    + * indicating the true value of number of updates for a blob. This node helps to maintain sanity in case
    + * leadership changes due to crashing.
    + *
    + * 2. Delete does not require to hand over the sequence number.
    + *
    + * 3. Finally, the update has few scenarios.
    + *
    + *  The class implements a TreeSet. The basic idea is if all the nimbodes have the same
    + *  sequence number for the blob, then the number of elements in the set is 1 which holds
    + *  the latest value of sequence number. If the number of elements are greater than 1 then it
    + *  implies that there is sequence mismatch and there is need for syncing the blobs across
    + *  nimbodes.
    + *
    + *  The logic for handing over sequence numbers based on the state are described as follows
    + *  Here consider Nimbus-1 alias as N1 and Nimbus-2 alias as N2.
    + *  Scenario 1:
    + *  Example: Normal create/update scenario
    + *  Operation     Nimbus-1:state     Nimbus-2:state     Seq-Num-Nimbus-1  Seq-Num-Nimbus-2          Max-Seq-Num
    + *  Create-Key1   alive - Leader     alive              1                                           1
    + *  Sync          alive - Leader     alive              1                 1 (callback -> download)  1
    + *  Update-Key1   alive - Leader     alive              2                 1                         2
    + *  Sync          alive - Leader     alive              2                 2 (callback -> download)  2
    + *
    + *  Scenario 2:
    + *  Example: Leader nimbus crash followed by leader election, update and ex-leader restored again
    + *  Operation     Nimbus-1:state     Nimbus-2:state     Seq-Num-Nimbus-1  Seq-Num-Nimbus-2          Max-Seq-Num
    + *  Create        alive - Leader     alive              1                                           1
    + *  Sync          alive - Leader     alive              1                 1 (callback -> download)  1
    + *  Update        alive - Leader     alive              2                 1                         2
    + *  Sync          alive - Leader     alive              2                 2 (callback -> download)  2
    + *  Update        alive - Leader     alive              3                 2                         3
    + *  Crash         crash - Leader     alive              3                 2                         3
    + *  New - Leader  crash              alive - Leader     3 (Invalid)       2                         3
    + *  Update        crash              alive - Leader     3 (Invalid)       4 (max-seq-num + 1)       4
    + *  N1-Restored   alive              alive - Leader     0                 4                         4
    + *  Sync          alive              alive - Leader     4                 4                         4
    + *
    + *  Scenario 3:
    + *  Example: Leader nimbus crash followed by leader election, update and ex-leader restored again
    + *  Operation     Nimbus-1:state     Nimbus-2:state     Seq-Num-Nimbus-1  Seq-Num-Nimbus-2          Max-Seq-Num
    + *  Create        alive - Leader     alive              1                                           1
    + *  Sync          alive - Leader     alive              1                 1 (callback -> download)  1
    + *  Update        alive - Leader     alive              2                 1                         2
    + *  Sync          alive - Leader     alive              2                 2 (callback -> download)  2
    + *  Update        alive - Leader     alive              3                 2                         3
    + *  Crash         crash - Leader     alive              3                 2                         3
    + *  Elect Leader  crash              alive - Leader     3 (Invalid)       2                         3
    + *  N1-Restored   alive              alive - Leader     3                 2                         3
    + *  Read/Update   alive              alive - Leader     3                 4 (Downloads from N1)     4
    + *  Sync          alive              alive - Leader     4 (callback)      4                         4
    + *  Here the download is triggered whenever an operation corresponding to the blob is triggered on the
    + *  nimbus like a read or update operation. Here, in the read/update call it is hard to know which call
    + *  is read or update. Hence, by incrementing the sequence number to max-seq-num + 1 we ensure that the
    + *  synchronization happens appropriately and all nimbodes have the same blob.
    + */
    +public class KeySequenceNumber {
    +    private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
    +    private final String BLOBSTORE_SUBTREE="/blobstore";
    +    private final String BLOBSTORE_MAX_KEY_SEQUENCE_SUBTREE="/blobstoremaxkeysequencenumber";
    +    private final String key;
    +    private final NimbusInfo nimbusInfo;
    +    private final int INT_CAPACITY = 4;
    +
    +    public KeySequenceNumber(String key, NimbusInfo nimbusInfo) {
    +        this.key = key;
    +        this.nimbusInfo = nimbusInfo;
    +    }
    +
    +    public int getKeySequenceNumber(Map conf) {
    +        TreeSet<Integer> sequenceNumbers = new TreeSet<Integer>();
    +        CuratorFramework zkClient = BlobStoreUtils.createZKClient(conf);
    +        try {
    +            // Key has not been created yet and it is the first time it is being created
    +            if(zkClient.checkExists().forPath(BLOBSTORE_SUBTREE + "/" + key) == null) {
    +                zkClient.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT)
    +                        .withACL(ZooDefs.Ids.OPEN_ACL_UNSAFE).forPath(BLOBSTORE_MAX_KEY_SEQUENCE_SUBTREE + "/" + key);
    +                zkClient.setData().forPath(BLOBSTORE_MAX_KEY_SEQUENCE_SUBTREE + "/" + key,
    +                        ByteBuffer.allocate(INT_CAPACITY).putInt(1).array());
    +                return 1;
    +            }
    +
    +            // When all nimbodes go down and one or few of them come up
    +            // Unfortunately there might not be an exact way to know which one contains the most updated blob,
    +            // if all go down which is unlikely. Hence there might be a need to update the blob if all go down.
    +            List<String> stateInfoList = zkClient.getChildren().forPath(BLOBSTORE_SUBTREE + "/" + key);
    +            LOG.debug("stateInfoList-size {} stateInfoList-data {}", stateInfoList.size(), stateInfoList);
    +            if(stateInfoList.isEmpty()) {
    +                return getMaxSequenceNumber(zkClient);
    +            }
    +
    +            LOG.debug("stateInfoSize {}", stateInfoList.size());
    +            // In all other cases check for the latest update sequence of the blob on the nimbus
    +            // and assign the appropriate number. Check if all are have same sequence number,
    +            // if not assign the highest sequence number.
    +            for (String stateInfo:stateInfoList) {
    +                sequenceNumbers.add(Integer.parseInt(BlobStoreUtils.normalizeVersionInfo(stateInfo)[1]));
    +            }
    +
    +            // Update scenario 2 and 3 explain the code logic written here
    +            // especially when nimbus crashes and comes up after and before update
    +            // respectively.
    +            int currentSeqNumber = getMaxSequenceNumber(zkClient);
    +            if (!checkIfStateContainsCurrentNimbusHost(stateInfoList, nimbusInfo) && !nimbusInfo.isLeader()) {
    +                if (sequenceNumbers.last() < currentSeqNumber) {
    +                    return currentSeqNumber;
    +                } else {
    +                    return 0;
    +                }
    +            }
    +
    +            // It covers scenario's expalined in scenario 3 when nimbus-1 holding the latest
    +            // update goes down before it is downloaded by nimbus-2. Nimbus-2 gets elected as a leader
    +            // after which nimbus-1 comes back up and a read or update is performed.
    +            if (!checkIfStateContainsCurrentNimbusHost(stateInfoList, nimbusInfo) && nimbusInfo.isLeader()) {
    +                incrementMaxSequenceNumber(zkClient, currentSeqNumber);
    +                return currentSeqNumber + 1;
    +            }
    +
    +            // This code logic covers the update scenarios in 2 when the nimbus-1 goes down
    +            // before syncing the blob to nimbus-2 and an update happens.
    +            // If seq-num for nimbus-2 is 2 and max-seq-number is 3 then next sequence number is 4
    +            // (max-seq-number + 1).
    +            // Other scenario it covers is when max-seq-number and nimbus seq number are equal.
    +            if (sequenceNumbers.size() == 1) {
    +                if (sequenceNumbers.first() < currentSeqNumber) {
    +                    incrementMaxSequenceNumber(zkClient, currentSeqNumber);
    +                    return currentSeqNumber + 1;
    +                } else {
    +                    incrementMaxSequenceNumber(zkClient, currentSeqNumber);
    +                    return sequenceNumbers.first() + 1;
    +                }
    +            }
    +        } catch(Exception e) {
    +            LOG.error("Exception {}", e);
    +        } finally {
    +            if (zkClient != null) {
    +                zkClient.close();
    +            }
    +        }
    +        // Normal create update sync scenario returns the greatest sequence number in the set
    +        return sequenceNumbers.last();
    +    }
    +
    +    private boolean checkIfStateContainsCurrentNimbusHost(List<String> stateInfoList, NimbusInfo nimbusInfo) {
    +        boolean containsNimbusHost = false;
    +        for(String stateInfo:stateInfoList) {
    +            if(stateInfo.contains(nimbusInfo.getHost())) {
    +                containsNimbusHost = true;
    +                break;
    +            }
    +        }
    +        return containsNimbusHost;
    +    }
    +
    +    private void incrementMaxSequenceNumber(CuratorFramework zkClient, int count) throws Exception {
    +        zkClient.setData().forPath(BLOBSTORE_MAX_KEY_SEQUENCE_SUBTREE + "/" + key,
    +                ByteBuffer.allocate(INT_CAPACITY).putInt(count++).array());
    --- End diff --
    
    `count++` this is a post-increment operator.  This will write `count`'s value that was given by the caller, then increment the value, then throw it away.  It looks like we want `count+1` here instead.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45271866
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStore.java ---
    @@ -0,0 +1,446 @@
    +/**
    + * 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 java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.NoSuchElementException;
    +import java.util.Set;
    +import java.util.regex.Pattern;
    +
    +import javax.security.auth.Subject;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import backtype.storm.daemon.Shutdownable;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +import backtype.storm.generated.SettableBlobMeta;
    +
    +/**
    + * Provides a way to store blobs that can be downloaded.
    + * Blobs must be able to be uploaded and listed from Nimbus,
    + * and downloaded from the Supervisors. It is a key value based
    + * store. Key being a string and value being the blob data.
    + *
    + * ACL checking must take place against the provided subject.
    + * If the blob store does not support Security it must validate
    + * that all ACLs set are always WORLD, everything.
    + *
    + * The users can upload their blobs through the blob store command
    + * line utility. The command line utilty also allows us to update,
    + * delete.
    + *
    + * Modifying the replication factor only works for HdfsBlobStore
    + * as for the LocalFsBlobStore the replication is dependent on
    + * the number of Nimbodes available.
    + */
    +public abstract class BlobStore implements Shutdownable {
    +  public static final Logger LOG = LoggerFactory.getLogger(BlobStore.class);
    +  private static final Pattern KEY_PATTERN = Pattern.compile("^[\\w \\t\\.:_-]+$");
    +  protected static final String BASE_BLOBS_DIR_NAME = "blobs";
    +
    +  /**
    +   * Allows us to initialize the blob store
    +   * @param conf
    +   * @param baseDir
    +   */
    --- End diff --
    
    missing a param in docstring.  If we want these we should give descriptions. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45121513
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -1376,8 +1514,8 @@
                   (check-storm-active! nimbus storm-name false)
                   ;;cred-update-lock is not needed here because creds are being added for the first time.
                   (.set-credentials! storm-cluster-state storm-id credentials storm-conf)
    -              (setup-storm-code nimbus conf storm-id uploadedJarLocation storm-conf topology)
    -              (.setup-code-distributor! storm-cluster-state storm-id (:nimbus-host-port-info nimbus))
    +              (log-message "uploadedJar" uploadedJarLocation)
    --- End diff --
    
    minor: Can we add a space to make the log message cleaner?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r44995171
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/supervisor.clj ---
    @@ -326,16 +331,29 @@
                                              (log-error t "Error when processing event")
                                              (exit-process! 20 "Error when processing an event")
                                              ))
    +   :blob-update-timer (mk-timer :kill-fn (fn [t]
    +                                           (log-error t "Error when processing blob-update")
    +                                           (exit-process! 20 "Error when processing a blob-update")
    +                                           ))
    --- End diff --
    
    Danglers


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by unsleepy22 <gi...@git.apache.org>.
Github user unsleepy22 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46023509
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobSynchronizer.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * 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.blobstore.BlobStoreUtils;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashSet;
    +import java.util.Map;
    +import java.util.Set;;
    +
    +/**
    + * Is called periodically and updates the nimbus with blobs based on the state stored inside the zookeeper
    + * for a non leader nimbus trying to be in sync with the operations performed on the leader nimbus.
    + */
    +public class BlobSynchronizer {
    +  private static final Logger LOG = LoggerFactory.getLogger(BlobSynchronizer.class);
    +  private CuratorFramework zkClient;
    +  private Map conf;
    +  private BlobStore blobStore;
    +  private Set<String> blobStoreKeySet = new HashSet<String>();
    +  private Set<String> zookeeperKeySet = new HashSet<String>();
    +  private NimbusInfo nimbusInfo;
    +
    +  public BlobSynchronizer(BlobStore blobStore, Map conf) {
    +    this.blobStore = blobStore;
    +    this.conf = conf;
    +  }
    +
    +  public void setNimbusInfo(NimbusInfo nimbusInfo) {
    +    this.nimbusInfo = nimbusInfo;
    +  }
    +
    +  public void setZookeeperKeySet(Set<String> zookeeperKeySet) {
    +    this.zookeeperKeySet = zookeeperKeySet;
    +  }
    +
    +  public void setBlobStoreKeySet(Set<String> blobStoreKeySet) {
    +    this.blobStoreKeySet = blobStoreKeySet;
    +  }
    +
    +  public Set<String> getBlobStoreKeySet() {
    +    Set<String> keySet = new HashSet<String>();
    +    keySet.addAll(blobStoreKeySet);
    +    return keySet;
    +  }
    +
    +  public Set<String> getZookeeperKeySet() {
    +    Set<String> keySet = new HashSet<String>();
    +    keySet.addAll(zookeeperKeySet);
    +    return keySet;
    +  }
    +
    +  public synchronized void syncBlobs() {
    +    try {
    +    LOG.debug("Sync blobs - blobstore keys {}, zookeeper keys {}",getBlobStoreKeySet(), getZookeeperKeySet());
    +    zkClient = BlobStoreUtils.createZKClient(conf);
    +    deleteKeySetFromBlobStoreNotOnZookeeper(getBlobStoreKeySet(), getZookeeperKeySet());
    +    updateKeySetForBlobStore(getBlobStoreKeySet());
    +    Set<String> keySetToDownload = getKeySetToDownload(getBlobStoreKeySet(), getZookeeperKeySet());
    +    LOG.debug("Key set Blobstore-> Zookeeper-> DownloadSet {}-> {}-> {}", getBlobStoreKeySet(), getZookeeperKeySet(), keySetToDownload);
    +
    +    for (String key : keySetToDownload) {
    +      Set<NimbusInfo> nimbusInfoSet = BlobStoreUtils.getNimbodesWithLatestSequenceNumberOfBlob(zkClient, key);
    +      if(BlobStoreUtils.downloadMissingBlob(conf, blobStore, key, nimbusInfoSet)) {
    +          BlobStoreUtils.createStateInZookeeper(conf, key, nimbusInfo);
    +      }
    +    }
    +    if (zkClient !=null) {
    +      zkClient.close();
    +    }
    +    } catch(InterruptedException exp) {
    +        LOG.error("InterruptedException {}", exp);
    +    } catch(Exception exp) {
    +        throw new RuntimeException(exp);
    +    }
    +  }
    +
    +  public void deleteKeySetFromBlobStoreNotOnZookeeper(Set<String> keySetBlobStore, Set<String> keySetZookeeper) throws Exception {
    +    if (keySetBlobStore.removeAll(keySetZookeeper)
    +            || (keySetZookeeper.isEmpty() && !keySetBlobStore.isEmpty())) {
    +      LOG.debug("Key set to delete in blobstore {}", keySetBlobStore);
    +      for (String key : keySetBlobStore) {
    +        blobStore.deleteBlob(key, BlobStoreUtils.getNimbusSubject());
    +      }
    +    }
    +  }
    +
    +  // Update current key list inside the blobstore if the version changes
    +  public void updateKeySetForBlobStore(Set<String> keySetBlobStore) {
    +    try {
    +      for (String key : keySetBlobStore) {
    +        LOG.debug("updating blob");
    +          BlobStoreUtils.updateKeyForBlobStore(conf, blobStore, zkClient, key, nimbusInfo);
    --- End diff --
    
    not aligned indent


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45760787
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/KeyVersion.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * 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.zookeeper.CreateMode;
    +import org.apache.zookeeper.ZooDefs;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.TreeSet;
    +import java.util.Map;
    +import java.util.List;
    +
    +/**
    + * Class hands over the version of the key to be stored within the zookeeper
    + */
    +public class KeyVersion {
    +  private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
    +  private final String BLOBSTORE_SUBTREE="/blobstore";
    +  private final String BLOBSTORE_KEY_COUNTER_SUBTREE="/blobstorekeycounter";
    +  private String key;
    +  private NimbusInfo nimbusInfo;
    +
    +  public KeyVersion(String key, NimbusInfo nimbusInfo) {
    +    this.key = key;
    +    this.nimbusInfo = nimbusInfo;
    +  }
    +
    +  public int getKeyVersion(Map conf) {
    +    TreeSet<Integer> versions = new TreeSet<Integer>();
    +    CuratorFramework zkClient = Utils.createZKClient(conf);
    +    try {
    +      // Key has not been created yet and it is the first time it is being created
    +      if(zkClient.checkExists().forPath(BLOBSTORE_SUBTREE + "/" + key) == null) {
    +        zkClient.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT)
    +                .withACL(ZooDefs.Ids.OPEN_ACL_UNSAFE).forPath(BLOBSTORE_KEY_COUNTER_SUBTREE + "/" + key + "/" + 1);
    +        return 1;
    +      }
    +
    +      // When all nimbodes go down and one or few of them come up
    +      // Unfortunately there might not be an exact way to know which one contains the most updated blob
    +      // if all go down which is unlikely. Hence there might be a need to update the blob if all go down
    +      List<String> stateInfoList = zkClient.getChildren().forPath(BLOBSTORE_SUBTREE + "/" + key);
    +      LOG.debug("stateInfoList {} stateInfoList {}", stateInfoList.size(), stateInfoList);
    +      if(stateInfoList.isEmpty()) {
    +        return getKeyVersionCounterValue(zkClient, key);
    +      }
    +
    +      LOG.debug("stateInfoSize {}", stateInfoList.size());
    +      // In all other cases check for the latest version on the nimbus and assign the version
    +      // check if all are have same version, if not assign the highest version
    +      for (String stateInfo:stateInfoList) {
    +        versions.add(Integer.parseInt(Utils.normalizeVersionInfo(stateInfo)[1]));
    +      }
    +
    +      int currentCounter = getKeyVersionCounterValue(zkClient, key);
    +      // This condition returns version when a nimbus crashes and comes up
    +      if (!checkIfStateContainsCurrentNimbusHost(stateInfoList, nimbusInfo) && !nimbusInfo.isLeader()) {
    +        if (versions.last() < currentCounter) {
    +          return currentCounter;
    +        } else {
    +          return currentCounter - 1;
    +        }
    +      }
    +      // Condition checks for an update scenario
    +      if (stateInfoList.size() >= 1 && versions.size() == 1) {
    +        if (versions.first() < getKeyVersionCounterValue(zkClient, key)) {
    +          incrementCounter(zkClient, key, currentCounter);
    +          return currentCounter + 1;
    +        } else {
    +          incrementCounter(zkClient, key, currentCounter);
    +          return versions.first() + 1;
    +        }
    +      }
    +    } catch(Exception e) {
    +      LOG.error("Exception {}", e);
    +    } finally {
    +      if (zkClient != null) {
    +        zkClient.close();
    +      }
    +    }
    +    return versions.last();
    +  }
    +
    +  public boolean checkIfStateContainsCurrentNimbusHost(List<String> stateInfoList, NimbusInfo nimbusInfo) {
    +    boolean containsNimbusHost = false;
    +    for(String stateInfo:stateInfoList) {
    +      if(stateInfo.contains(nimbusInfo.getHost())) {
    +        containsNimbusHost = true;
    +        break;
    +      }
    +    }
    +    return containsNimbusHost;
    +  }
    +
    +  public void incrementCounter(CuratorFramework zkClient, String key, int count) throws Exception {
    +    zkClient.delete().deletingChildrenIfNeeded().forPath(BLOBSTORE_KEY_COUNTER_SUBTREE + "/" + key);
    +    zkClient.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT)
    +            .withACL(ZooDefs.Ids.OPEN_ACL_UNSAFE).forPath(BLOBSTORE_KEY_COUNTER_SUBTREE + "/" + key + "/" + (count + 1));
    +  }
    +
    +  public int getKeyVersionCounterValue(CuratorFramework zkClient, String key) throws Exception {
    --- End diff --
    
    @d2r addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by redsanket <gi...@git.apache.org>.
Github user redsanket commented on the pull request:

    https://github.com/apache/storm/pull/845#issuecomment-157180530
  
    storm.thrift:
    
    enum AccessControlType {
      OTHER = 1,
      USER = 2
      //eventually ,GROUP=3
    }
    Would it be better to remove this comment and create a JIRA for adding group
    support instead? In the JIRA, we could point to this spot in the code as a
    hint.
    
    @derek -> https://issues.apache.org/jira/browse/STORM-1166


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43779998
  
    --- Diff: conf/defaults.yaml ---
    @@ -63,7 +63,7 @@ nimbus.supervisor.timeout.secs: 60
     nimbus.monitor.freq.secs: 10
     nimbus.cleanup.inbox.freq.secs: 600
     nimbus.inbox.jar.expiration.secs: 3600
    -nimbus.code.sync.freq.secs: 300
    +nimbus.code.sync.freq.secs: 120
    --- End diff --
    
    What is the reason for this change?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by unsleepy22 <gi...@git.apache.org>.
Github user unsleepy22 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r46024256
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStore.java ---
    @@ -0,0 +1,296 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.utils.Utils;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.security.auth.Subject;
    +import java.io.ByteArrayOutputStream;
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.FileNotFoundException;
    +import java.io.InputStream;
    +
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;;
    +
    +import static backtype.storm.blobstore.BlobStoreAclHandler.ADMIN;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.READ;
    +import static backtype.storm.blobstore.BlobStoreAclHandler.WRITE;
    +
    +/**
    + * Provides a local file system backed blob store implementation for Nimbus.
    + */
    +public class LocalFsBlobStore extends BlobStore {
    +  public static final Logger LOG = LoggerFactory.getLogger(LocalFsBlobStore.class);
    +  private static final String DATA_PREFIX = "data_";
    +  private static final String META_PREFIX = "meta_";
    +  protected BlobStoreAclHandler _aclHandler;
    +  private final String BLOBSTORE_SUBTREE = "/blobstore/";
    +  private NimbusInfo nimbusInfo;
    +  private FileBlobStoreImpl fbs;
    +  private Map conf;
    +
    +  @Override
    +  public void prepare(Map conf, String overrideBase, NimbusInfo nimbusInfo) {
    +    this.conf = conf;
    +    this.nimbusInfo = nimbusInfo;
    +     if (overrideBase == null) {
    +       overrideBase = (String)conf.get(Config.BLOBSTORE_DIR);
    +      if (overrideBase == null) {
    +        overrideBase = (String) conf.get(Config.STORM_LOCAL_DIR);
    +      }
    +    }
    +    File baseDir = new File(overrideBase, BASE_BLOBS_DIR_NAME);
    +    try {
    +      fbs = new FileBlobStoreImpl(baseDir, conf);
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +    _aclHandler = new BlobStoreAclHandler(conf);
    +  }
    +
    +  @Override
    +  public AtomicOutputStream createBlob(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException {
    +    LOG.debug("Creating Blob for key {}", key);
    +    validateKey(key);
    +    _aclHandler.normalizeSettableBlobMeta(key, meta, who, READ | WRITE | ADMIN);
    +    BlobStoreAclHandler.validateSettableACLs(key, meta.get_acl());
    +    _aclHandler.hasPermissions(meta.get_acl(), READ | WRITE | ADMIN, who, key);
    +    if (fbs.exists(DATA_PREFIX+key)) {
    +      throw new KeyAlreadyExistsException(key);
    +    }
    +    BlobStoreFileOutputStream mOut = null;
    +    try {
    +      mOut = new BlobStoreFileOutputStream(fbs.write(META_PREFIX+key, true));
    +      mOut.write(Utils.thriftSerialize(meta));
    +      mOut.close();
    +      mOut = null;
    +      return new BlobStoreFileOutputStream(fbs.write(DATA_PREFIX+key, true));
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    } finally {
    +      if (mOut != null) {
    +        try {
    +          mOut.cancel();
    +        } catch (IOException e) {
    +          //Ignored
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public AtomicOutputStream updateBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    validateKey(key);
    +    checkForBlobOrDownload(key);
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(meta.get_acl(), WRITE, who, key);
    +    try {
    +      return new BlobStoreFileOutputStream(fbs.write(DATA_PREFIX+key, false));
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  private SettableBlobMeta getStoredBlobMeta(String key) throws KeyNotFoundException {
    +    InputStream in = null;
    +    try {
    +      LocalFsBlobStoreFile pf = fbs.read(META_PREFIX+key);
    +      try {
    +        in = pf.getInputStream();
    +      } catch (FileNotFoundException fnf) {
    +        throw new KeyNotFoundException(key);
    +      }
    +      ByteArrayOutputStream out = new ByteArrayOutputStream();
    +      byte [] buffer = new byte[2048];
    +      int len;
    +      while ((len = in.read(buffer)) > 0) {
    +        out.write(buffer, 0, len);
    +      }
    +      in.close();
    +      in = null;
    +      return Utils.thriftDeserialize(SettableBlobMeta.class, out.toByteArray());
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    } finally {
    +      if (in != null) {
    +        try {
    +          in.close();
    +        } catch (IOException e) {
    +          //Ignored
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public ReadableBlobMeta getBlobMeta(String key, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    validateKey(key);
    +    if(!checkForBlobOrDownload(key)) {
    +      checkForBlobUpdate(key);
    +    }
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.validateUserCanReadMeta(meta.get_acl(), who, key);
    +    ReadableBlobMeta rbm = new ReadableBlobMeta();
    +    rbm.set_settable(meta);
    +    try {
    +      LocalFsBlobStoreFile pf = fbs.read(DATA_PREFIX+key);
    +      rbm.set_version(pf.getModTime());
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +    return rbm;
    +  }
    +
    +  @Override
    +  public void setBlobMeta(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    validateKey(key);
    +    checkForBlobOrDownload(key);
    +    _aclHandler.normalizeSettableBlobMeta(key, meta, who, ADMIN);
    +    BlobStoreAclHandler.validateSettableACLs(key, meta.get_acl());
    +    SettableBlobMeta orig = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(orig.get_acl(), ADMIN, who, key);
    +    BlobStoreFileOutputStream mOut = null;
    +    try {
    +      mOut = new BlobStoreFileOutputStream(fbs.write(META_PREFIX+key, false));
    +      mOut.write(Utils.thriftSerialize(meta));
    +      mOut.close();
    +      mOut = null;
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    } finally {
    +      if (mOut != null) {
    +        try {
    +          mOut.cancel();
    +        } catch (IOException e) {
    +          //Ignored
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void deleteBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    validateKey(key);
    +    checkForBlobOrDownload(key);
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(meta.get_acl(), WRITE, who, key);
    +    try {
    +      fbs.deleteKey(DATA_PREFIX+key);
    +      fbs.deleteKey(META_PREFIX+key);
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override
    +  public InputStreamWithMeta getBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException {
    +    validateKey(key);
    +    if(!checkForBlobOrDownload(key)) {
    +      checkForBlobUpdate(key);
    +    }
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(meta.get_acl(), READ, who, key);
    +    try {
    +      return new BlobStoreFileInputStream(fbs.read(DATA_PREFIX+key));
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override
    +  public Iterator<String> listKeys() {
    +    try {
    +      return new KeyTranslationIterator(fbs.listKeys(), DATA_PREFIX);
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    +    }
    +  }
    +
    +  @Override
    +  public void shutdown() {
    +  }
    +
    +  @Override
    +  public int getBlobReplication(String key, Subject who) throws Exception {
    +    validateKey(key);
    +    SettableBlobMeta meta = getStoredBlobMeta(key);
    +    _aclHandler.hasPermissions(meta.get_acl(), READ, who, key);
    +    CuratorFramework zkClient = BlobStoreUtils.createZKClient(conf);
    +    if (zkClient.checkExists().forPath(BLOBSTORE_SUBTREE + key) == null) {
    +       zkClient.close();
    --- End diff --
    
    I'd prefer to add a try... finally block  to ensure `zkClient.close` is called.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45121741
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -1661,16 +1799,16 @@
                         (.set_isLeader nimbus-summary (and (= leader-host (.get_host nimbus-summary)) (= leader-port (.get_port nimbus-summary))))))
     
                   topology-summaries (dofor [[id base] bases :when base]
    -	                                  (let [assignment (.assignment-info storm-cluster-state id nil)
    +                                         (let [assignment (.assignment-info storm-cluster-state id nil)
    --- End diff --
    
    revert?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r45800279
  
    --- Diff: storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStore.java ---
    @@ -0,0 +1,297 @@
    +/**
    + * 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.Config;
    +import backtype.storm.generated.SettableBlobMeta;
    +import backtype.storm.generated.AuthorizationException;
    +import backtype.storm.generated.KeyAlreadyExistsException;
    +import backtype.storm.generated.KeyNotFoundException;
    +import backtype.storm.generated.ReadableBlobMeta;
    +
    +import backtype.storm.nimbus.NimbusInfo;
    +import backtype.storm.utils.Utils;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.thrift.TBase;
    --- End diff --
    
    Still unused.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by knusbaum <gi...@git.apache.org>.
Github user knusbaum commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43785495
  
    --- Diff: storm-core/src/clj/backtype/storm/command/blobstore.clj ---
    @@ -0,0 +1,166 @@
    +;; 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.
    +(ns backtype.storm.command.blobstore
    +  (:import [java.io InputStream OutputStream])
    +  (:use [backtype.storm config])
    +  (:import [backtype.storm.generated SettableBlobMeta AccessControl AuthorizationException
    +            KeyNotFoundException])
    +  (:import [backtype.storm.blobstore BlobStoreAclHandler])
    +  (:use [clojure.string :only [split]])
    +  (:use [clojure.tools.cli :only [cli]])
    +  (:use [clojure.java.io :only [copy input-stream output-stream]])
    +  (:use [backtype.storm blobstore log util])
    +  (:gen-class))
    +
    +(defn update-blob-from-stream
    +  "Update a blob in the blob store from an InputStream"
    +  [key ^InputStream in]
    +  (with-configured-blob-client blobstore
    +    (let [out (.updateBlob blobstore key)]
    +      (try 
    +        (copy in out)
    +        (.close out)
    +        (.createStateInZookeeper blobstore key)
    +        (catch Exception e
    +          (log-message e)
    +          (.cancel out)
    +          (throw e))))))
    +
    +(defn create-blob-from-stream
    +  "Create a blob in the blob storm from an InputStream"
    --- End diff --
    
    "blob storm"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43799348
  
    --- Diff: pom.xml ---
    @@ -646,11 +650,92 @@
                     <version>${thrift.version}</version>
                     <scope>compile</scope>
                 </dependency>
    +			<!-- used by examples/storm-starter -->
                 <dependency>
                     <groupId>junit</groupId>
    -                <artifactId>junit</artifactId>
    +		        <artifactId>junit</artifactId>
                     <version>${junit.version}</version>
    +		        <scope>test</scope>
    +            </dependency>
    --- End diff --
    
    Some white-space issue here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-876] Blobstore API

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/845#discussion_r43939413
  
    --- Diff: storm-core/src/clj/backtype/storm/daemon/nimbus.clj ---
    @@ -346,53 +400,104 @@
           [(.getNodeId slot) (.getPort slot)]
           )))
     
    +(defn- get-metadata-version [blob-store key subject]
    +  (let [blob-meta (.getBlobMeta blob-store key subject)]
    +    (.get_version blob-meta)))
    +
    +(defn get-key-set-from-blob-store [blob-store]
    +  (let [key-iter (.listKeys blob-store get-nimbus-subject)]
    +    (set (iterator-seq key-iter))))
    +
     (defn- setup-storm-code [nimbus conf storm-id tmp-jar-location storm-conf topology]
    -  (let [stormroot (master-stormdist-root conf storm-id)]
    -   (log-message "nimbus file location:" stormroot)
    -   (FileUtils/forceMkdir (File. stormroot))
    -   (FileUtils/cleanDirectory (File. stormroot))
    -   (setup-jar conf tmp-jar-location stormroot)
    -   (FileUtils/writeByteArrayToFile (File. (master-stormcode-path stormroot)) (Utils/serialize topology))
    -   (FileUtils/writeByteArrayToFile (File. (master-stormconf-path stormroot)) (Utils/toCompressedJsonConf storm-conf))
    -   (if (:code-distributor nimbus) (.upload (:code-distributor nimbus) stormroot storm-id))
    -   ))
    +  (let [subject (get-subject)
    +        storm-cluster-state (:storm-cluster-state nimbus)
    +        blob-store (:blob-store nimbus)
    +        jar-key (master-stormjar-key storm-id)
    +        code-key (master-stormcode-key storm-id)
    +        conf-key (master-stormconf-key storm-id)
    +        nimbus-host-port-info (:nimbus-host-port-info nimbus)]
    +    (if tmp-jar-location ;;in local mode there is no jar
    +      (do
    +        (.createBlob blob-store jar-key (FileInputStream. tmp-jar-location) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +        (if (instance? LocalFsBlobStore blob-store)
    +          (.setup-blobstore! storm-cluster-state jar-key nimbus-host-port-info (get-metadata-version blob-store jar-key subject)))))
    +    (.createBlob blob-store conf-key (Utils/toCompressedJsonConf storm-conf) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state conf-key nimbus-host-port-info (get-metadata-version blob-store conf-key subject)))
    +    (.createBlob blob-store code-key (Utils/serialize topology) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
    +    (if (instance? LocalFsBlobStore blob-store)
    +      (.setup-blobstore! storm-cluster-state code-key nimbus-host-port-info (get-metadata-version blob-store code-key subject)))))
    +
    +(defn- read-storm-topology [storm-id blob-store]
    +  (Utils/deserialize
    +    (.readBlob blob-store (master-stormcode-key storm-id) (get-subject)) StormTopology))
    +
    +(defn- get-blob-replication-count [blob-key nimbus]
    +  (if (:blob-store nimbus)
    +          (-> (:blob-store nimbus)
    +            (.getBlobReplication  blob-key get-nimbus-subject)
    +            (.get_replication))))
     
     (defn- wait-for-desired-code-replication [nimbus conf storm-id]
       (let [min-replication-count (conf TOPOLOGY-MIN-REPLICATION-COUNT)
             max-replication-wait-time (conf TOPOLOGY-MAX-REPLICATION-WAIT-TIME-SEC)
    -        total-wait-time (atom 0)
    -        current-replication-count (atom (if (:code-distributor nimbus) (.getReplicationCount (:code-distributor nimbus) storm-id) 0))]
    -  (if (:code-distributor nimbus)
    -    (while (and (> min-replication-count @current-replication-count)
    -             (or (= -1 max-replication-wait-time)
    -               (< @total-wait-time max-replication-wait-time)))
    +        current-replication-count-jar (if (not (local-mode? conf)) (atom
    +                                                                     (get-blob-replication-count (master-stormjar-key storm-id) nimbus))
    +                                                                     (atom min-replication-count))
    --- End diff --
    
    Conditional here is hard to read the way it is formatted:
    
    Maybe something like this?
    ```Clojure
    jar-replication-count (get-blob-replication-count
                             (master-stormjar-key storm-id)
                             nimbus)
    current-replication-count-jar (if (not (local-mode? conf))
                                      (atom jar-replication-count)
                                      (atom min-replication-count))
    ```



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---