You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by bo...@apache.org on 2015/12/04 16:03:56 UTC

[01/17] storm git commit: Blobstore API STORM- 876

Repository: storm
Updated Branches:
  refs/heads/master ce2d49b92 -> 6babbb08c


http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/test/jvm/backtype/storm/blobstore/ClientBlobStoreTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/blobstore/ClientBlobStoreTest.java b/storm-core/test/jvm/backtype/storm/blobstore/ClientBlobStoreTest.java
new file mode 100644
index 0000000..17a8f71
--- /dev/null
+++ b/storm-core/test/jvm/backtype/storm/blobstore/ClientBlobStoreTest.java
@@ -0,0 +1,179 @@
+/**
+ * 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.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 org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+public class ClientBlobStoreTest {
+
+  private ClientBlobStore client;
+  public class TestClientBlobStore extends ClientBlobStore {
+
+    private Map<String, SettableBlobMeta> allBlobs;
+    @Override
+    public void prepare(Map conf) {
+      this.conf = conf;
+      allBlobs = new HashMap<String, SettableBlobMeta>();
+    }
+
+    @Override
+    protected AtomicOutputStream createBlobToExtend(String key, SettableBlobMeta meta) throws AuthorizationException, KeyAlreadyExistsException {
+      allBlobs.put(key, meta);
+      return null;
+    }
+
+    @Override
+    public AtomicOutputStream updateBlob(String key) throws AuthorizationException, KeyNotFoundException {
+      return null;
+    }
+
+    @Override
+    public ReadableBlobMeta getBlobMeta(String key) throws AuthorizationException, KeyNotFoundException {
+      ReadableBlobMeta reableMeta = null;
+      if (allBlobs.containsKey(key)) {
+        reableMeta = new ReadableBlobMeta();
+        reableMeta.set_settable(allBlobs.get(key));
+      }
+      return reableMeta;
+    }
+
+    @Override
+    protected void setBlobMetaToExtend(String key, SettableBlobMeta meta) throws AuthorizationException, KeyNotFoundException {
+    }
+
+    @Override
+    public void deleteBlob(String key) throws AuthorizationException, KeyNotFoundException {
+    }
+
+    @Override
+    public InputStreamWithMeta getBlob(String key) throws AuthorizationException, KeyNotFoundException {
+      return null;
+    }
+
+    @Override
+    public Iterator<String> listKeys() {
+      return null;
+    }
+
+    @Override
+    public void shutdown() {
+    }
+
+    @Override
+    public int getBlobReplication(String key) {
+      return -1;
+    }
+
+    @Override
+    public int updateBlobReplication(String key, int replication) {
+      return -1;
+    }
+
+    @Override
+    public boolean setClient(Map conf, NimbusClient client) {
+      return false;
+    }
+
+    @Override
+    public void createStateInZookeeper(String key) {
+    }
+  }
+
+  @Before
+  public void setUp() throws Exception {
+
+    client = new TestClientBlobStore();
+    Map conf = new HashMap<String,String>();
+    client.prepare(conf);
+
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    client = null;
+  }
+
+  @Test(expected=AuthorizationException.class)
+  public void testDuplicateACLsForCreate() throws Exception {
+    SettableBlobMeta meta = new SettableBlobMeta();
+    AccessControl submitterAcl = BlobStoreAclHandler.parseAccessControl("u:tester:rwa");
+    meta.add_to_acl(submitterAcl);
+    AccessControl duplicateAcl = BlobStoreAclHandler.parseAccessControl("u:tester:r--");
+    meta.add_to_acl(duplicateAcl);
+    String testKey = "testDuplicateACLsBlobKey";
+    client.createBlob(testKey, meta);
+  }
+
+  @Test
+  public void testGoodACLsForCreate() throws Exception {
+    SettableBlobMeta meta = new SettableBlobMeta();
+    AccessControl submitterAcl = BlobStoreAclHandler.parseAccessControl("u:tester:rwa");
+    meta.add_to_acl(submitterAcl);
+    String testKey = "testBlobKey";
+    client.createBlob(testKey, meta);
+    validatedBlobAcls(testKey);
+  }
+
+  @Test(expected=AuthorizationException.class)
+  public void testDuplicateACLsForSetBlobMeta() throws Exception {
+    String testKey = "testDuplicateACLsBlobKey";
+    SettableBlobMeta meta = new SettableBlobMeta();
+    createTestBlob(testKey, meta);
+    AccessControl duplicateAcl = BlobStoreAclHandler.parseAccessControl("u:tester:r--");
+    meta.add_to_acl(duplicateAcl);
+    client.setBlobMeta(testKey, meta);
+  }
+
+  @Test
+  public void testGoodACLsForSetBlobMeta() throws Exception {
+    String testKey = "testBlobKey";
+    SettableBlobMeta meta = new SettableBlobMeta();
+    createTestBlob(testKey, meta);
+    meta.add_to_acl(BlobStoreAclHandler.parseAccessControl("u:nextuser:r--"));
+    client.setBlobMeta(testKey,meta);
+    validatedBlobAcls(testKey);
+  }
+
+  private void createTestBlob(String testKey, SettableBlobMeta meta) throws AuthorizationException, KeyAlreadyExistsException {
+    AccessControl submitterAcl = BlobStoreAclHandler.parseAccessControl("u:tester:rwa");
+    meta.add_to_acl(submitterAcl);
+    client.createBlob(testKey, meta);
+  }
+
+  private void validatedBlobAcls(String testKey) throws KeyNotFoundException, AuthorizationException {
+    ReadableBlobMeta blobMeta = client.getBlobMeta(testKey);
+    Assert.assertNotNull("The blob" + testKey + "does not have any readable blobMeta.", blobMeta);
+    SettableBlobMeta settableBlob = blobMeta.get_settable();
+    Assert.assertNotNull("The blob" + testKey + "does not have any settable blobMeta.", settableBlob);
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/test/jvm/backtype/storm/localizer/LocalizedResourceRetentionSetTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/localizer/LocalizedResourceRetentionSetTest.java b/storm-core/test/jvm/backtype/storm/localizer/LocalizedResourceRetentionSetTest.java
new file mode 100644
index 0000000..15f3da3
--- /dev/null
+++ b/storm-core/test/jvm/backtype/storm/localizer/LocalizedResourceRetentionSetTest.java
@@ -0,0 +1,85 @@
+/**
+ * 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.localizer;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
+
+public class LocalizedResourceRetentionSetTest {
+
+  @Test
+  public void testAddResources() throws Exception {
+    LocalizedResourceRetentionSet lrretset = new LocalizedResourceRetentionSet(10);
+    LocalizedResourceSet lrset = new LocalizedResourceSet("user1");
+    LocalizedResource localresource1 = new LocalizedResource("key1", "testfile1", false, "topo1");
+    LocalizedResource localresource2 = new LocalizedResource("key2", "testfile2", false, "topo1");
+    // check adding reference to local resource with topology of same name
+    localresource2.addReference(("topo2"));
+
+    lrset.addResource("key1", localresource1, false);
+    lrset.addResource("key2", localresource2, false);
+    lrretset.addResources(lrset);
+    assertEquals("number to clean is not 0", 0, lrretset.getSizeWithNoReferences());
+    localresource1.removeReference(("topo1"));
+    lrretset.addResources(lrset);
+    assertEquals("number to clean is not 1", 1, lrretset.getSizeWithNoReferences());
+    localresource2.removeReference(("topo1"));
+    lrretset.addResources(lrset);
+    assertEquals("number to clean is not 1", 1, lrretset.getSizeWithNoReferences());
+    localresource2.removeReference(("topo2"));
+    lrretset.addResources(lrset);
+    assertEquals("number to clean is not 2", 2, lrretset.getSizeWithNoReferences());
+  }
+
+  @Test
+  public void testCleanup() throws Exception {
+    LocalizedResourceRetentionSet lrretset = spy(new LocalizedResourceRetentionSet(10));
+    LocalizedResourceSet lrset = new LocalizedResourceSet("user1");
+    // no reference to key1
+    LocalizedResource localresource1 = new LocalizedResource("key1", "testfile1", false);
+    localresource1.setSize(10);
+    // no reference to archive1
+    LocalizedResource archiveresource1 = new LocalizedResource("archive1", "testarchive1", true);
+    archiveresource1.setSize(20);
+    // reference to key2
+    LocalizedResource localresource2 = new LocalizedResource("key2", "testfile2", false, "topo1");
+    // check adding reference to local resource with topology of same name
+    localresource2.addReference(("topo1"));
+    localresource2.setSize(10);
+    lrset.addResource("key1", localresource1, false);
+    lrset.addResource("key2", localresource2, false);
+    lrset.addResource("archive1", archiveresource1, true);
+
+    lrretset.addResources(lrset);
+    assertEquals("number to clean is not 2", 2, lrretset.getSizeWithNoReferences());
+
+    // shouldn't change number since file doesn't exist and delete fails
+    lrretset.cleanup();
+    assertEquals("resource cleaned up", 2, lrretset.getSizeWithNoReferences());
+
+    // make deleteResource return true even though file doesn't exist
+    when(lrretset.deleteResource(localresource1)).thenReturn(true);
+    when(lrretset.deleteResource(localresource2)).thenReturn(true);
+    when(lrretset.deleteResource(archiveresource1)).thenReturn(true);
+    lrretset.cleanup();
+    assertEquals("resource not cleaned up", 0, lrretset.getSizeWithNoReferences());
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/test/jvm/backtype/storm/localizer/LocalizedResourceSetTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/localizer/LocalizedResourceSetTest.java b/storm-core/test/jvm/backtype/storm/localizer/LocalizedResourceSetTest.java
new file mode 100644
index 0000000..839113f
--- /dev/null
+++ b/storm-core/test/jvm/backtype/storm/localizer/LocalizedResourceSetTest.java
@@ -0,0 +1,74 @@
+/**
+ * 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.localizer;
+
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class LocalizedResourceSetTest {
+
+  private final String user1 = "user1";
+
+  @Test
+  public void testGetUser() throws Exception {
+    LocalizedResourceSet lrset = new LocalizedResourceSet(user1);
+    assertEquals("user is wrong", user1, lrset.getUser());
+  }
+
+  @Test
+  public void testGetSize() throws Exception {
+    LocalizedResourceSet lrset = new LocalizedResourceSet(user1);
+    LocalizedResource localresource1 = new LocalizedResource("key1", "testfile1", false, "topo1");
+    LocalizedResource localresource2 = new LocalizedResource("key2", "testfile2", true, "topo1");
+    assertEquals("size is wrong", 0, lrset.getSize());
+    lrset.addResource("key1", localresource1, false);
+    assertEquals("size is wrong", 1, lrset.getSize());
+    lrset.addResource("key2", localresource2, true);
+    assertEquals("size is wrong", 2, lrset.getSize());
+  }
+
+  @Test
+  public void testGet() throws Exception {
+    LocalizedResourceSet lrset = new LocalizedResourceSet(user1);
+    LocalizedResource localresource1 = new LocalizedResource("key1", "testfile1", false, "topo1");
+    LocalizedResource localresource2 = new LocalizedResource("key2", "testfile2", true, "topo1");
+    lrset.addResource("key1", localresource1, false);
+    lrset.addResource("key2", localresource2, true);
+    assertEquals("get doesn't return same object", localresource1, lrset.get("key1", false));
+    assertEquals("get doesn't return same object", localresource2, lrset.get("key2", true));
+
+  }
+
+  @Test
+  public void testExists() throws Exception {
+    LocalizedResourceSet lrset = new LocalizedResourceSet(user1);
+    LocalizedResource localresource1 = new LocalizedResource("key1", "testfile1", false, "topo1");
+    LocalizedResource localresource2 = new LocalizedResource("key2", "testfile2", true, "topo1");
+    lrset.addResource("key1", localresource1, false);
+    lrset.addResource("key2", localresource2, true);
+    assertEquals("doesn't exist", true, lrset.exists("key1", false));
+    assertEquals("doesn't exist", true, lrset.exists("key2", true));
+    boolean val = lrset.remove(localresource1);
+    assertTrue("remove failed", val);
+    assertEquals("does exist", false, lrset.exists("key1", false));
+    assertEquals("doesn't exist", true, lrset.exists("key2", true));
+    val = lrset.remove(localresource1);
+    assertFalse("remove success when shouldn't have been", val);
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/test/jvm/backtype/storm/localizer/LocalizerTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/localizer/LocalizerTest.java b/storm-core/test/jvm/backtype/storm/localizer/LocalizerTest.java
new file mode 100644
index 0000000..b47a3b4
--- /dev/null
+++ b/storm-core/test/jvm/backtype/storm/localizer/LocalizerTest.java
@@ -0,0 +1,671 @@
+/**
+ * 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.localizer;
+
+import backtype.storm.Config;
+import backtype.storm.blobstore.BlobStoreAclHandler;
+import backtype.storm.blobstore.ClientBlobStore;
+import backtype.storm.blobstore.InputStreamWithMeta;
+import backtype.storm.blobstore.LocalFsBlobStore;
+import backtype.storm.generated.AccessControl;
+import backtype.storm.generated.AccessControlType;
+import backtype.storm.generated.AuthorizationException;
+import backtype.storm.generated.KeyNotFoundException;
+import backtype.storm.generated.ReadableBlobMeta;
+import backtype.storm.generated.SettableBlobMeta;
+import backtype.storm.utils.Utils;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.util.*;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+
+public class LocalizerTest {
+
+  private File baseDir;
+
+  private final String user1 = "user1";
+  private final String user2 = "user2";
+  private final String user3 = "user3";
+
+  ClientBlobStore mockblobstore = mock(ClientBlobStore.class);
+
+
+  class TestLocalizer extends Localizer {
+
+    TestLocalizer(Map conf, String baseDir) {
+      super(conf, baseDir);
+    }
+
+    @Override
+    protected ClientBlobStore getClientBlobStore() {
+      return mockblobstore;
+    }
+  }
+
+  class TestInputStreamWithMeta extends InputStreamWithMeta {
+    private InputStream iostream;
+    private byte[] buffer = null;
+    private int offset = 0;
+    private int end = 0;
+    private boolean eof = false;
+
+    public TestInputStreamWithMeta() {
+      iostream = IOUtils.toInputStream("some test data for my input stream");
+    }
+
+    public TestInputStreamWithMeta(InputStream istream) {
+       iostream = istream;
+    }
+
+    @Override
+    public long getVersion() throws IOException {
+      return 1;
+    }
+
+    @Override
+    public synchronized int read() {
+      return 0;
+    }
+
+    @Override
+    public synchronized int read(byte[] b)
+    throws IOException {
+      int length = iostream.read(b);
+      if (length == 0) {
+        return -1;
+      }
+      return length;
+    }
+
+    @Override
+    public long getFileLength() {
+        return 0;
+    }
+  };
+
+  @Before
+  public void setUp() throws Exception {
+    baseDir = new File("/tmp/blob-store-localizer-test-"+ UUID.randomUUID());
+    if (!baseDir.mkdir()) {
+      throw new IOException("failed to create base directory");
+    }
+    ClientBlobStore mockblobstore = mock(ClientBlobStore.class);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    FileUtils.deleteDirectory(baseDir);
+  }
+
+  public String constructUserCacheDir(String base, String user) {
+    return base + "/" + Localizer.USERCACHE + "/" + user;
+  }
+
+  public String constructExpectedFilesDir(String base, String user) {
+    return constructUserCacheDir(base, user) + "/" + Localizer.FILECACHE + "/" + Localizer.FILESDIR;
+  }
+
+  public String constructExpectedArchivesDir(String base, String user) {
+    return constructUserCacheDir(base, user) + "/" + Localizer.FILECACHE + "/" + Localizer
+        .ARCHIVESDIR;
+  }
+
+  @Test
+  public void testDirPaths() throws Exception {
+    Map conf = new HashMap();
+    Localizer localizer = new TestLocalizer(conf, baseDir.toString());
+
+    String expectedDir = constructUserCacheDir(baseDir.toString(), user1);
+    assertEquals("get local user dir doesn't return right value",
+        expectedDir, localizer.getLocalUserDir(user1).toString());
+
+    String expectedFileDir = expectedDir + "/" + Localizer.FILECACHE;
+    assertEquals("get local user file dir doesn't return right value",
+        expectedFileDir, localizer.getLocalUserFileCacheDir(user1).toString());
+  }
+
+  @Test
+  public void testReconstruct() throws Exception {
+    Map conf = new HashMap();
+
+    String expectedFileDir1 = constructExpectedFilesDir(baseDir.toString(), user1);
+    String expectedArchiveDir1 = constructExpectedArchivesDir(baseDir.toString(), user1);
+    String expectedFileDir2 = constructExpectedFilesDir(baseDir.toString(), user2);
+    String expectedArchiveDir2 = constructExpectedArchivesDir(baseDir.toString(), user2);
+
+    String key1 = "testfile1.txt";
+    String key2 = "testfile2.txt";
+    String key3 = "testfile3.txt";
+    String key4 = "testfile4.txt";
+
+    String archive1 = "archive1";
+    String archive2 = "archive2";
+
+    File user1file1 = new File(expectedFileDir1, key1 + Utils.DEFAULT_CURRENT_BLOB_SUFFIX);
+    File user1file2 = new File(expectedFileDir1, key2 + Utils.DEFAULT_CURRENT_BLOB_SUFFIX);
+    File user2file3 = new File(expectedFileDir2, key3 + Utils.DEFAULT_CURRENT_BLOB_SUFFIX);
+    File user2file4 = new File(expectedFileDir2, key4 + Utils.DEFAULT_CURRENT_BLOB_SUFFIX);
+
+    File user1archive1 = new File(expectedArchiveDir1, archive1 + Utils.DEFAULT_CURRENT_BLOB_SUFFIX);
+    File user2archive2 = new File(expectedArchiveDir2, archive2 + Utils.DEFAULT_CURRENT_BLOB_SUFFIX);
+    File user1archive1file = new File(user1archive1, "file1");
+    File user2archive2file = new File(user2archive2, "file2");
+
+    // setup some files/dirs to emulate supervisor restart
+    assertTrue("Failed setup filecache dir1", new File(expectedFileDir1).mkdirs());
+    assertTrue("Failed setup filecache dir2", new File(expectedFileDir2).mkdirs());
+    assertTrue("Failed setup file1", user1file1.createNewFile());
+    assertTrue("Failed setup file2", user1file2.createNewFile());
+    assertTrue("Failed setup file3", user2file3.createNewFile());
+    assertTrue("Failed setup file4", user2file4.createNewFile());
+    assertTrue("Failed setup archive dir1", user1archive1.mkdirs());
+    assertTrue("Failed setup archive dir2", user2archive2.mkdirs());
+    assertTrue("Failed setup file in archivedir1", user1archive1file.createNewFile());
+    assertTrue("Failed setup file in archivedir2", user2archive2file.createNewFile());
+
+    Localizer localizer = new TestLocalizer(conf, baseDir.toString());
+
+    ArrayList<LocalResource> arrUser1Keys = new ArrayList<LocalResource>();
+    arrUser1Keys.add(new LocalResource(key1, false));
+    arrUser1Keys.add(new LocalResource(archive1, true));
+    localizer.addReferences(arrUser1Keys, user1, "topo1");
+
+    LocalizedResourceSet lrsrcSet = localizer.getUserResources().get(user1);
+    assertEquals("local resource set size wrong", 3, lrsrcSet.getSize());
+    assertEquals("user doesn't match", user1, lrsrcSet.getUser());
+    LocalizedResource key1rsrc = lrsrcSet.get(key1, false);
+    assertNotNull("Local resource doesn't exist but should", key1rsrc);
+    assertEquals("key doesn't match", key1, key1rsrc.getKey());
+    assertEquals("refcount doesn't match", 1, key1rsrc.getRefCount());
+    LocalizedResource key2rsrc = lrsrcSet.get(key2, false);
+    assertNotNull("Local resource doesn't exist but should", key2rsrc);
+    assertEquals("key doesn't match", key2, key2rsrc.getKey());
+    assertEquals("refcount doesn't match", 0, key2rsrc.getRefCount());
+    LocalizedResource archive1rsrc = lrsrcSet.get(archive1, true);
+    assertNotNull("Local resource doesn't exist but should", archive1rsrc);
+    assertEquals("key doesn't match", archive1, archive1rsrc.getKey());
+    assertEquals("refcount doesn't match", 1, archive1rsrc.getRefCount());
+
+    LocalizedResourceSet lrsrcSet2 = localizer.getUserResources().get(user2);
+    assertEquals("local resource set size wrong", 3, lrsrcSet2.getSize());
+    assertEquals("user doesn't match", user2, lrsrcSet2.getUser());
+    LocalizedResource key3rsrc = lrsrcSet2.get(key3, false);
+    assertNotNull("Local resource doesn't exist but should", key3rsrc);
+    assertEquals("key doesn't match", key3, key3rsrc.getKey());
+    assertEquals("refcount doesn't match", 0, key3rsrc.getRefCount());
+    LocalizedResource key4rsrc = lrsrcSet2.get(key4, false);
+    assertNotNull("Local resource doesn't exist but should", key4rsrc);
+    assertEquals("key doesn't match", key4, key4rsrc.getKey());
+    assertEquals("refcount doesn't match", 0, key4rsrc.getRefCount());
+    LocalizedResource archive2rsrc = lrsrcSet2.get(archive2, true);
+    assertNotNull("Local resource doesn't exist but should", archive2rsrc);
+    assertEquals("key doesn't match", archive2, archive2rsrc.getKey());
+    assertEquals("refcount doesn't match", 0, archive2rsrc.getRefCount());
+  }
+
+  @Test
+  public void testArchivesTgz() throws Exception {
+    testArchives("test/jvm/backtype/storm/localizer/localtestwithsymlink.tgz", true,
+        21344);
+  }
+
+  @Test
+  public void testArchivesZip() throws Exception {
+    testArchives("test/jvm/backtype/storm/localizer/localtest.zip", false,
+        21348);
+  }
+
+  @Test
+  public void testArchivesTarGz() throws Exception {
+    testArchives("test/jvm/backtype/storm/localizer/localtestwithsymlink.tar.gz",
+        true, 21344);
+  }
+
+  @Test
+  public void testArchivesTar() throws Exception {
+    testArchives("test/jvm/backtype/storm/localizer/localtestwithsymlink.tar", true,
+        21344);
+  }
+
+  @Test
+  public void testArchivesJar() throws Exception {
+    testArchives("test/jvm/backtype/storm/localizer/localtestwithsymlink.jar", false,
+        21416);
+  }
+
+  // archive passed in must contain symlink named tmptestsymlink is not a zip file
+  public void testArchives(String archivePath, boolean supportSymlinks, int size) throws Exception {
+    Map conf = new HashMap();
+    // set clean time really high so doesn't kick in
+    conf.put(Config.SUPERVISOR_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS, 60*60*1000);
+
+    String key1 = new File(archivePath).getName();
+    String topo1 = "topo1";
+    Localizer localizer = new TestLocalizer(conf, baseDir.toString());
+    // set really small so will do cleanup
+    localizer.setTargetCacheSize(1);
+
+    ReadableBlobMeta rbm = new ReadableBlobMeta();
+    rbm.set_settable(new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING));
+    when(mockblobstore.getBlobMeta(key1)).thenReturn(rbm);
+
+    when(mockblobstore.getBlob(key1)).thenReturn(new TestInputStreamWithMeta(new
+        FileInputStream(archivePath)));
+
+    long timeBefore = System.nanoTime();
+    File user1Dir = localizer.getLocalUserFileCacheDir(user1);
+    assertTrue("failed to create user dir", user1Dir.mkdirs());
+    LocalizedResource lrsrc = localizer.getBlob(new LocalResource(key1, true), user1, topo1,
+        user1Dir);
+    long timeAfter = System.nanoTime();
+
+    String expectedUserDir = baseDir + "/" + Localizer.USERCACHE + "/" + user1;
+    String expectedFileDir = expectedUserDir + "/" + Localizer.FILECACHE + "/" + Localizer.ARCHIVESDIR;
+    assertTrue("user filecache dir not created", new File(expectedFileDir).exists());
+    File keyFile = new File(expectedFileDir, key1 + ".0");
+    assertTrue("blob not created", keyFile.exists());
+    assertTrue("blob is not uncompressed", keyFile.isDirectory());
+    File symlinkFile = new File(keyFile, "tmptestsymlink");
+
+    if (supportSymlinks) {
+      assertTrue("blob uncompressed doesn't contain symlink", Files.isSymbolicLink(
+          symlinkFile.toPath()));
+    } else {
+      assertTrue("blob symlink file doesn't exist", symlinkFile.exists());
+    }
+
+    LocalizedResourceSet lrsrcSet = localizer.getUserResources().get(user1);
+    assertEquals("local resource set size wrong", 1, lrsrcSet.getSize());
+    assertEquals("user doesn't match", user1, lrsrcSet.getUser());
+    LocalizedResource key1rsrc = lrsrcSet.get(key1, true);
+    assertNotNull("Local resource doesn't exist but should", key1rsrc);
+    assertEquals("key doesn't match", key1, key1rsrc.getKey());
+    assertEquals("refcount doesn't match", 1, key1rsrc.getRefCount());
+    assertEquals("file path doesn't match", keyFile.toString(), key1rsrc.getFilePathWithVersion());
+    assertEquals("size doesn't match", size, key1rsrc.getSize());
+    assertTrue("timestamp not within range", (key1rsrc.getLastAccessTime() >= timeBefore && key1rsrc
+        .getLastAccessTime() <= timeAfter));
+
+    timeBefore = System.nanoTime();
+    localizer.removeBlobReference(lrsrc.getKey(), user1, topo1, true);
+    timeAfter = System.nanoTime();
+
+    lrsrcSet = localizer.getUserResources().get(user1);
+    assertEquals("local resource set size wrong", 1, lrsrcSet.getSize());
+    key1rsrc = lrsrcSet.get(key1, true);
+    assertNotNull("Local resource doesn't exist but should", key1rsrc);
+    assertEquals("refcount doesn't match", 0, key1rsrc.getRefCount());
+    assertTrue("timestamp not within range", (key1rsrc.getLastAccessTime() >= timeBefore && key1rsrc
+        .getLastAccessTime() <= timeAfter));
+
+    // should remove the blob since cache size set really small
+    localizer.handleCacheCleanup();
+
+    lrsrcSet = localizer.getUserResources().get(user1);
+    assertFalse("blob contents not deleted", symlinkFile.exists());
+    assertFalse("blob not deleted", keyFile.exists());
+    assertFalse("blob file dir not deleted", new File(expectedFileDir).exists());
+    assertFalse("blob dir not deleted", new File(expectedUserDir).exists());
+    assertNull("user set should be null", lrsrcSet);
+
+  }
+
+
+  @Test
+  public void testBasic() throws Exception {
+    Map conf = new HashMap();
+    // set clean time really high so doesn't kick in
+    conf.put(Config.SUPERVISOR_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS, 60*60*1000);
+
+    String key1 = "key1";
+    String topo1 = "topo1";
+    Localizer localizer = new TestLocalizer(conf, baseDir.toString());
+    // set really small so will do cleanup
+    localizer.setTargetCacheSize(1);
+
+    ReadableBlobMeta rbm = new ReadableBlobMeta();
+    rbm.set_settable(new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING));
+    when(mockblobstore.getBlobMeta(key1)).thenReturn(rbm);
+
+    when(mockblobstore.getBlob(key1)).thenReturn(new TestInputStreamWithMeta());
+
+    long timeBefore = System.nanoTime();
+    File user1Dir = localizer.getLocalUserFileCacheDir(user1);
+    assertTrue("failed to create user dir", user1Dir.mkdirs());
+    LocalizedResource lrsrc = localizer.getBlob(new LocalResource(key1, false), user1, topo1,
+        user1Dir);
+    long timeAfter = System.nanoTime();
+
+    String expectedUserDir = baseDir + "/" + Localizer.USERCACHE + "/" + user1;
+    String expectedFileDir = expectedUserDir + "/" + Localizer.FILECACHE + "/" + Localizer.FILESDIR;
+    assertTrue("user filecache dir not created", new File(expectedFileDir).exists());
+    File keyFile = new File(expectedFileDir, key1);
+    File keyFileCurrentSymlink = new File(expectedFileDir, key1 + Utils.DEFAULT_CURRENT_BLOB_SUFFIX);
+
+    assertTrue("blob not created", keyFileCurrentSymlink.exists());
+
+    LocalizedResourceSet lrsrcSet = localizer.getUserResources().get(user1);
+    assertEquals("local resource set size wrong", 1, lrsrcSet.getSize());
+    assertEquals("user doesn't match", user1, lrsrcSet.getUser());
+    LocalizedResource key1rsrc = lrsrcSet.get(key1, false);
+    assertNotNull("Local resource doesn't exist but should", key1rsrc);
+    assertEquals("key doesn't match", key1, key1rsrc.getKey());
+    assertEquals("refcount doesn't match", 1, key1rsrc.getRefCount());
+    assertEquals("file path doesn't match", keyFile.toString(), key1rsrc.getFilePath());
+    assertEquals("size doesn't match", 34, key1rsrc.getSize());
+    assertTrue("timestamp not within range", (key1rsrc.getLastAccessTime() >= timeBefore && key1rsrc
+        .getLastAccessTime() <= timeAfter));
+
+    timeBefore = System.nanoTime();
+    localizer.removeBlobReference(lrsrc.getKey(), user1, topo1, false);
+    timeAfter = System.nanoTime();
+
+    lrsrcSet = localizer.getUserResources().get(user1);
+    assertEquals("local resource set size wrong", 1, lrsrcSet.getSize());
+    key1rsrc = lrsrcSet.get(key1, false);
+    assertNotNull("Local resource doesn't exist but should", key1rsrc);
+    assertEquals("refcount doesn't match", 0, key1rsrc.getRefCount());
+    assertTrue("timestamp not within range", (key1rsrc.getLastAccessTime() >= timeBefore && key1rsrc
+        .getLastAccessTime() <= timeAfter));
+
+    // should remove the blob since cache size set really small
+    localizer.handleCacheCleanup();
+
+    lrsrcSet = localizer.getUserResources().get(user1);
+    assertNull("user set should be null", lrsrcSet);
+    assertFalse("blob not deleted", keyFile.exists());
+    assertFalse("blob dir not deleted", new File(expectedFileDir).exists());
+    assertFalse("blob dir not deleted", new File(expectedUserDir).exists());
+  }
+
+  @Test
+  public void testMultipleKeysOneUser() throws Exception {
+    Map conf = new HashMap();
+    // set clean time really high so doesn't kick in
+    conf.put(Config.SUPERVISOR_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS, 60*60*1000);
+
+    String key1 = "key1";
+    String topo1 = "topo1";
+    String key2 = "key2";
+    String key3 = "key3";
+    Localizer localizer = new TestLocalizer(conf, baseDir.toString());
+    // set to keep 2 blobs (each of size 34)
+    localizer.setTargetCacheSize(68);
+
+    ReadableBlobMeta rbm = new ReadableBlobMeta();
+    rbm.set_settable(new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING));
+    when(mockblobstore.getBlobMeta(anyString())).thenReturn(rbm);
+    when(mockblobstore.getBlob(key1)).thenReturn(new TestInputStreamWithMeta());
+    when(mockblobstore.getBlob(key2)).thenReturn(new TestInputStreamWithMeta());
+    when(mockblobstore.getBlob(key3)).thenReturn(new TestInputStreamWithMeta());
+
+    List<LocalResource> keys = Arrays.asList(new LocalResource[]{new LocalResource(key1, false),
+        new LocalResource(key2, false), new LocalResource(key3, false)});
+    File user1Dir = localizer.getLocalUserFileCacheDir(user1);
+    assertTrue("failed to create user dir", user1Dir.mkdirs());
+
+    List<LocalizedResource> lrsrcs = localizer.getBlobs(keys, user1, topo1, user1Dir);
+    LocalizedResource lrsrc = lrsrcs.get(0);
+    LocalizedResource lrsrc2 = lrsrcs.get(1);
+    LocalizedResource lrsrc3 = lrsrcs.get(2);
+
+    String expectedFileDir = baseDir + "/" + Localizer.USERCACHE + "/" + user1 +
+        "/" + Localizer.FILECACHE + "/" + Localizer.FILESDIR;
+    assertTrue("user filecache dir not created", new File(expectedFileDir).exists());
+    File keyFile = new File(expectedFileDir, key1 + Utils.DEFAULT_CURRENT_BLOB_SUFFIX);
+    File keyFile2 = new File(expectedFileDir, key2 + Utils.DEFAULT_CURRENT_BLOB_SUFFIX);
+    File keyFile3 = new File(expectedFileDir, key3 + Utils.DEFAULT_CURRENT_BLOB_SUFFIX);
+
+    assertTrue("blob not created", keyFile.exists());
+    assertTrue("blob not created", keyFile2.exists());
+    assertTrue("blob not created", keyFile3.exists());
+    assertEquals("size doesn't match", 34, keyFile.length());
+    assertEquals("size doesn't match", 34, keyFile2.length());
+    assertEquals("size doesn't match", 34, keyFile3.length());
+    assertEquals("size doesn't match", 34, lrsrc.getSize());
+    assertEquals("size doesn't match", 34, lrsrc3.getSize());
+    assertEquals("size doesn't match", 34, lrsrc2.getSize());
+
+    LocalizedResourceSet lrsrcSet = localizer.getUserResources().get(user1);
+    assertEquals("local resource set size wrong", 3, lrsrcSet.getSize());
+    assertEquals("user doesn't match", user1, lrsrcSet.getUser());
+
+    long timeBefore = System.nanoTime();
+    localizer.removeBlobReference(lrsrc.getKey(), user1, topo1, false);
+    localizer.removeBlobReference(lrsrc2.getKey(), user1, topo1, false);
+    localizer.removeBlobReference(lrsrc3.getKey(), user1, topo1, false);
+    long timeAfter = System.nanoTime();
+
+    // add reference to one and then remove reference again so it has newer timestamp
+    lrsrc = localizer.getBlob(new LocalResource(key1, false), user1, topo1, user1Dir);
+    assertTrue("timestamp not within range", (lrsrc.getLastAccessTime() >= timeBefore && lrsrc
+        .getLastAccessTime() <= timeAfter));
+    localizer.removeBlobReference(lrsrc.getKey(), user1, topo1, false);
+
+    // should remove the second blob first
+    localizer.handleCacheCleanup();
+
+    lrsrcSet = localizer.getUserResources().get(user1);
+    assertEquals("local resource set size wrong", 2, lrsrcSet.getSize());
+    assertFalse("blob not deleted", keyFile2.exists());
+    assertTrue("blob deleted", keyFile.exists());
+    assertTrue("blob deleted", keyFile3.exists());
+
+    // set size to cleanup another one
+    localizer.setTargetCacheSize(34);
+
+    // should remove the third blob
+    localizer.handleCacheCleanup();
+
+    lrsrcSet = localizer.getUserResources().get(user1);
+    assertEquals("local resource set size wrong", 1, lrsrcSet.getSize());
+    assertTrue("blob deleted", keyFile.exists());
+    assertFalse("blob not deleted", keyFile3.exists());
+  }
+
+  @Test(expected = AuthorizationException.class)
+  public void testFailAcls() throws Exception {
+    Map conf = new HashMap();
+    // set clean time really high so doesn't kick in
+    conf.put(Config.SUPERVISOR_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS, 60 * 60 * 1000);
+
+    String topo1 = "topo1";
+    String key1 = "key1";
+    Localizer localizer = new TestLocalizer(conf, baseDir.toString());
+
+    ReadableBlobMeta rbm = new ReadableBlobMeta();
+    // set acl so user doesn't have read access
+    AccessControl acl = new AccessControl(AccessControlType.USER, BlobStoreAclHandler.ADMIN);
+    acl.set_name(user1);
+    rbm.set_settable(new SettableBlobMeta(Arrays.asList(acl)));
+    when(mockblobstore.getBlobMeta(anyString())).thenReturn(rbm);
+    when(mockblobstore.getBlob(key1)).thenReturn(new TestInputStreamWithMeta());
+    File user1Dir = localizer.getLocalUserFileCacheDir(user1);
+    assertTrue("failed to create user dir", user1Dir.mkdirs());
+
+    // This should throw AuthorizationException because auth failed
+    localizer.getBlob(new LocalResource(key1, false), user1, topo1, user1Dir);
+  }
+
+  @Test(expected = KeyNotFoundException.class)
+  public void testKeyNotFoundException() throws Exception {
+    Map conf = new HashMap();
+    String key1 = "key1";
+    conf.put(Config.STORM_LOCAL_DIR, "local");
+    conf.put(Config.BLOBSTORE_SUPERUSER, "superuser");
+    conf.put(Config.STORM_PRINCIPAL_TO_LOCAL_PLUGIN, "backtype.storm.security.auth.DefaultPrincipalToLocal");
+    LocalFsBlobStore bs = new LocalFsBlobStore();
+    LocalFsBlobStore spy = spy(bs);
+    Mockito.doReturn(true).when(spy).checkForBlobOrDownload(key1);
+    Mockito.doNothing().when(spy).checkForBlobUpdate(key1);
+    spy.prepare(conf,null,null);
+    spy.getBlob(key1, null);
+  }
+
+    @Test
+  public void testMultipleUsers() throws Exception {
+    Map conf = new HashMap();
+    // set clean time really high so doesn't kick in
+    conf.put(Config.SUPERVISOR_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS, 60*60*1000);
+
+    String topo1 = "topo1";
+    String topo2 = "topo2";
+    String topo3 = "topo3";
+    String key1 = "key1";
+    String key2 = "key2";
+    String key3 = "key3";
+    Localizer localizer = new TestLocalizer(conf, baseDir.toString());
+    // set to keep 2 blobs (each of size 34)
+    localizer.setTargetCacheSize(68);
+
+    ReadableBlobMeta rbm = new ReadableBlobMeta();
+    rbm.set_settable(new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING));
+    when(mockblobstore.getBlobMeta(anyString())).thenReturn(rbm);
+    when(mockblobstore.getBlob(key1)).thenReturn(new TestInputStreamWithMeta());
+    when(mockblobstore.getBlob(key2)).thenReturn(new TestInputStreamWithMeta());
+    when(mockblobstore.getBlob(key3)).thenReturn(new TestInputStreamWithMeta());
+
+    File user1Dir = localizer.getLocalUserFileCacheDir(user1);
+    assertTrue("failed to create user dir", user1Dir.mkdirs());
+    File user2Dir = localizer.getLocalUserFileCacheDir(user2);
+    assertTrue("failed to create user dir", user2Dir.mkdirs());
+    File user3Dir = localizer.getLocalUserFileCacheDir(user3);
+    assertTrue("failed to create user dir", user3Dir.mkdirs());
+
+    LocalizedResource lrsrc = localizer.getBlob(new LocalResource(key1, false), user1, topo1,
+        user1Dir);
+    LocalizedResource lrsrc2 = localizer.getBlob(new LocalResource(key2, false), user2, topo2,
+        user2Dir);
+    LocalizedResource lrsrc3 = localizer.getBlob(new LocalResource(key3, false), user3, topo3,
+        user3Dir);
+    // make sure we support different user reading same blob
+    LocalizedResource lrsrc1_user3 = localizer.getBlob(new LocalResource(key1, false), user3,
+        topo3, user3Dir);
+
+    String expectedUserDir1 = baseDir + "/" + Localizer.USERCACHE + "/" + user1;
+    String expectedFileDirUser1 = expectedUserDir1 + "/" + Localizer.FILECACHE + "/" +
+        Localizer.FILESDIR;
+    String expectedFileDirUser2 = baseDir + "/" + Localizer.USERCACHE + "/" + user2 +
+        "/" + Localizer.FILECACHE + "/" + Localizer.FILESDIR;
+    String expectedFileDirUser3 = baseDir + "/" + Localizer.USERCACHE + "/" + user3 +
+        "/" + Localizer.FILECACHE + "/" + Localizer.FILESDIR;
+    assertTrue("user filecache dir user1 not created", new File(expectedFileDirUser1).exists());
+    assertTrue("user filecache dir user2 not created", new File(expectedFileDirUser2).exists());
+    assertTrue("user filecache dir user3 not created", new File(expectedFileDirUser3).exists());
+
+    File keyFile = new File(expectedFileDirUser1, key1 + Utils.DEFAULT_CURRENT_BLOB_SUFFIX);
+    File keyFile2 = new File(expectedFileDirUser2, key2 + Utils.DEFAULT_CURRENT_BLOB_SUFFIX);
+    File keyFile3 = new File(expectedFileDirUser3, key3 + Utils.DEFAULT_CURRENT_BLOB_SUFFIX);
+    File keyFile1user3 = new File(expectedFileDirUser3, key1 + Utils.DEFAULT_CURRENT_BLOB_SUFFIX);
+
+    assertTrue("blob not created", keyFile.exists());
+    assertTrue("blob not created", keyFile2.exists());
+    assertTrue("blob not created", keyFile3.exists());
+    assertTrue("blob not created", keyFile1user3.exists());
+
+    LocalizedResourceSet lrsrcSet = localizer.getUserResources().get(user1);
+    assertEquals("local resource set size wrong", 1, lrsrcSet.getSize());
+    LocalizedResourceSet lrsrcSet2 = localizer.getUserResources().get(user2);
+    assertEquals("local resource set size wrong", 1, lrsrcSet2.getSize());
+    LocalizedResourceSet lrsrcSet3 = localizer.getUserResources().get(user3);
+    assertEquals("local resource set size wrong", 2, lrsrcSet3.getSize());
+
+    localizer.removeBlobReference(lrsrc.getKey(), user1, topo1, false);
+    // should remove key1
+    localizer.handleCacheCleanup();
+
+    lrsrcSet = localizer.getUserResources().get(user1);
+    lrsrcSet3 = localizer.getUserResources().get(user3);
+    assertNull("user set should be null", lrsrcSet);
+    assertFalse("blob dir not deleted", new File(expectedFileDirUser1).exists());
+    assertFalse("blob dir not deleted", new File(expectedUserDir1).exists());
+    assertEquals("local resource set size wrong", 2, lrsrcSet3.getSize());
+
+    assertTrue("blob deleted", keyFile2.exists());
+    assertFalse("blob not deleted", keyFile.exists());
+    assertTrue("blob deleted", keyFile3.exists());
+    assertTrue("blob deleted", keyFile1user3.exists());
+  }
+
+  @Test
+  public void testUpdate() throws Exception {
+    Map conf = new HashMap();
+    // set clean time really high so doesn't kick in
+    conf.put(Config.SUPERVISOR_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS, 60*60*1000);
+
+    String key1 = "key1";
+    String topo1 = "topo1";
+    String topo2 = "topo2";
+    Localizer localizer = new TestLocalizer(conf, baseDir.toString());
+
+    ReadableBlobMeta rbm = new ReadableBlobMeta();
+    rbm.set_version(1);
+    rbm.set_settable(new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING));
+    when(mockblobstore.getBlobMeta(key1)).thenReturn(rbm);
+    when(mockblobstore.getBlob(key1)).thenReturn(new TestInputStreamWithMeta());
+
+    File user1Dir = localizer.getLocalUserFileCacheDir(user1);
+    assertTrue("failed to create user dir", user1Dir.mkdirs());
+    LocalizedResource lrsrc = localizer.getBlob(new LocalResource(key1, false), user1, topo1,
+        user1Dir);
+
+    String expectedUserDir = baseDir + "/" + Localizer.USERCACHE + "/" + user1;
+    String expectedFileDir = expectedUserDir + "/" + Localizer.FILECACHE + "/" + Localizer.FILESDIR;
+    assertTrue("user filecache dir not created", new File(expectedFileDir).exists());
+    File keyFile = new File(expectedFileDir, key1);
+    File keyFileCurrentSymlink = new File(expectedFileDir, key1 + Utils.DEFAULT_CURRENT_BLOB_SUFFIX);
+    assertTrue("blob not created", keyFileCurrentSymlink.exists());
+    File versionFile = new File(expectedFileDir, key1 + Utils.DEFAULT_BLOB_VERSION_SUFFIX);
+    assertTrue("blob version file not created", versionFile.exists());
+    assertEquals("blob version not correct", 1, Utils.localVersionOfBlob(keyFile.toString()));
+
+    LocalizedResourceSet lrsrcSet = localizer.getUserResources().get(user1);
+    assertEquals("local resource set size wrong", 1, lrsrcSet.getSize());
+
+    // test another topology getting blob with updated version - it should update version now
+    rbm.set_version(2);
+
+    localizer.getBlob(new LocalResource(key1, false), user1, topo2, user1Dir);
+    assertTrue("blob version file not created", versionFile.exists());
+    assertEquals("blob version not correct", 2, Utils.localVersionOfBlob(keyFile.toString()));
+    assertTrue("blob file with version 2 not created", new File(keyFile + ".2").exists());
+
+    // now test regular updateBlob
+    rbm.set_version(3);
+
+    ArrayList<LocalResource> arr = new ArrayList<LocalResource>();
+    arr.add(new LocalResource(key1, false));
+    localizer.updateBlobs(arr, user1);
+    assertTrue("blob version file not created", versionFile.exists());
+    assertEquals("blob version not correct", 3, Utils.localVersionOfBlob(keyFile.toString()));
+    assertTrue("blob file with version 3 not created", new File(keyFile + ".3").exists());
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/test/jvm/backtype/storm/localizer/localtest.zip
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/localizer/localtest.zip b/storm-core/test/jvm/backtype/storm/localizer/localtest.zip
new file mode 100644
index 0000000..00ffd20
Binary files /dev/null and b/storm-core/test/jvm/backtype/storm/localizer/localtest.zip differ

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/test/jvm/backtype/storm/localizer/localtestwithsymlink.jar
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/localizer/localtestwithsymlink.jar b/storm-core/test/jvm/backtype/storm/localizer/localtestwithsymlink.jar
new file mode 100644
index 0000000..f60cc7b
Binary files /dev/null and b/storm-core/test/jvm/backtype/storm/localizer/localtestwithsymlink.jar differ

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/test/jvm/backtype/storm/localizer/localtestwithsymlink.tar
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/localizer/localtestwithsymlink.tar b/storm-core/test/jvm/backtype/storm/localizer/localtestwithsymlink.tar
new file mode 100644
index 0000000..caa316b
Binary files /dev/null and b/storm-core/test/jvm/backtype/storm/localizer/localtestwithsymlink.tar differ

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/test/jvm/backtype/storm/localizer/localtestwithsymlink.tar.gz
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/localizer/localtestwithsymlink.tar.gz b/storm-core/test/jvm/backtype/storm/localizer/localtestwithsymlink.tar.gz
new file mode 100644
index 0000000..e0f995f
Binary files /dev/null and b/storm-core/test/jvm/backtype/storm/localizer/localtestwithsymlink.tar.gz differ

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/test/jvm/backtype/storm/localizer/localtestwithsymlink.tgz
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/localizer/localtestwithsymlink.tgz b/storm-core/test/jvm/backtype/storm/localizer/localtestwithsymlink.tgz
new file mode 100644
index 0000000..0dd2134
Binary files /dev/null and b/storm-core/test/jvm/backtype/storm/localizer/localtestwithsymlink.tgz differ


[12/17] storm git commit: Blobstore API STORM- 876

Posted by bo...@apache.org.
Blobstore API STORM- 876


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

Branch: refs/heads/master
Commit: 7029aee576cff6d7159e9e1a991bb02aaf0d4cd8
Parents: 0acc1ce
Author: Sanket <sc...@untilservice-lm>
Authored: Mon Nov 30 14:56:15 2015 -0600
Committer: Sanket <sc...@untilservice-lm>
Committed: Mon Nov 30 14:56:15 2015 -0600

----------------------------------------------------------------------
 bin/storm.py                                    |    28 +-
 conf/defaults.yaml                              |    16 +-
 external/storm-hdfs/pom.xml                     |    70 +
 .../storm/hdfs/blobstore/HdfsBlobStore.java     |   370 +
 .../storm/hdfs/blobstore/HdfsBlobStoreFile.java |   196 +
 .../storm/hdfs/blobstore/HdfsBlobStoreImpl.java |   312 +
 .../hdfs/blobstore/HdfsClientBlobStore.java     |   115 +
 .../ha/codedistributor/HDFSCodeDistributor.java |   118 -
 .../storm/hdfs/blobstore/BlobStoreTest.java     |   530 +
 .../hdfs/blobstore/HdfsBlobStoreImplTest.java   |   231 +
 pom.xml                                         |    16 +-
 storm-core/pom.xml                              |    36 +-
 storm-core/src/clj/backtype/storm/blobstore.clj |    28 +
 storm-core/src/clj/backtype/storm/cluster.clj   |    92 +-
 .../cluster_state/zookeeper_state_factory.clj   |     4 +
 .../clj/backtype/storm/command/blobstore.clj    |   162 +
 storm-core/src/clj/backtype/storm/config.clj    |    24 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |   678 +-
 .../clj/backtype/storm/daemon/supervisor.clj    |   480 +-
 storm-core/src/clj/backtype/storm/testing.clj   |     5 +-
 storm-core/src/clj/backtype/storm/util.clj      |    16 +-
 storm-core/src/clj/backtype/storm/zookeeper.clj |    31 +-
 storm-core/src/jvm/backtype/storm/Config.java   |   123 +-
 .../storm/blobstore/AtomicOutputStream.java     |    32 +
 .../storm/blobstore/BlobKeySequenceInfo.java    |    40 +
 .../jvm/backtype/storm/blobstore/BlobStore.java |   445 +
 .../storm/blobstore/BlobStoreAclHandler.java    |   399 +
 .../backtype/storm/blobstore/BlobStoreFile.java |    50 +
 .../storm/blobstore/BlobStoreUtils.java         |   257 +
 .../storm/blobstore/BlobSynchronizer.java       |   124 +
 .../storm/blobstore/ClientBlobStore.java        |    62 +
 .../storm/blobstore/FileBlobStoreImpl.java      |   248 +
 .../storm/blobstore/InputStreamWithMeta.java    |    26 +
 .../jvm/backtype/storm/blobstore/KeyFilter.java |    22 +
 .../storm/blobstore/KeySequenceNumber.java      |   229 +
 .../storm/blobstore/LocalFsBlobStore.java       |   308 +
 .../storm/blobstore/LocalFsBlobStoreFile.java   |   159 +
 .../storm/blobstore/NimbusBlobStore.java        |   412 +
 .../backtype/storm/cluster/ClusterState.java    |     9 +
 .../storm/codedistributor/ICodeDistributor.java |    73 -
 .../LocalFileSystemCodeDistributor.java         |   123 -
 .../backtype/storm/generated/AccessControl.java |   627 +
 .../storm/generated/AccessControlType.java      |    62 +
 .../backtype/storm/generated/Assignment.java    |   244 +-
 .../storm/generated/BeginDownloadResult.java    |   608 +
 .../storm/generated/ClusterWorkerHeartbeat.java |    52 +-
 .../jvm/backtype/storm/generated/HBNodes.java   |    32 +-
 .../jvm/backtype/storm/generated/HBRecords.java |    36 +-
 .../generated/KeyAlreadyExistsException.java    |   406 +
 .../storm/generated/KeyNotFoundException.java   |   406 +
 .../storm/generated/LSApprovedWorkers.java      |    44 +-
 .../generated/LSSupervisorAssignments.java      |    48 +-
 .../backtype/storm/generated/LSTopoHistory.java |    64 +-
 .../storm/generated/LSTopoHistoryList.java      |    36 +-
 .../storm/generated/LSWorkerHeartbeat.java      |    36 +-
 .../storm/generated/ListBlobsResult.java        |   556 +
 .../storm/generated/LocalAssignment.java        |    36 +-
 .../storm/generated/LocalStateData.java         |    48 +-
 .../jvm/backtype/storm/generated/LogConfig.java |    48 +-
 .../jvm/backtype/storm/generated/Nimbus.java    | 26917 +++++++++++++----
 .../jvm/backtype/storm/generated/NodeInfo.java  |    32 +-
 .../storm/generated/ReadableBlobMeta.java       |   510 +
 .../storm/generated/SettableBlobMeta.java       |   567 +
 .../jvm/backtype/storm/generated/StormBase.java |    92 +-
 .../storm/generated/SupervisorInfo.java         |   152 +-
 .../storm/generated/TopologyHistoryInfo.java    |    32 +-
 .../backtype/storm/localizer/LocalResource.java |    44 +
 .../storm/localizer/LocalizedResource.java      |   130 +
 .../LocalizedResourceRetentionSet.java          |   140 +
 .../storm/localizer/LocalizedResourceSet.java   |   101 +
 .../jvm/backtype/storm/localizer/Localizer.java |   695 +
 .../storm/security/auth/NimbusPrincipal.java    |    29 +
 .../backtype/storm/utils/BufferInputStream.java |    53 +
 .../jvm/backtype/storm/utils/ShellUtils.java    |     7 +
 .../src/jvm/backtype/storm/utils/Utils.java     |   545 +-
 .../storm/validation/ConfigValidation.java      |    16 +-
 .../validation/ConfigValidationAnnotations.java |    11 +-
 storm-core/src/py/storm/Nimbus-remote           |    98 +
 storm-core/src/py/storm/Nimbus.py               |  5991 +++-
 storm-core/src/py/storm/ttypes.py               |   996 +-
 storm-core/src/storm.thrift                     |    59 +
 .../test/clj/backtype/storm/cluster_test.clj    |    20 +-
 .../test/clj/backtype/storm/nimbus_test.clj     |    43 +-
 .../storm/security/auth/ReqContext_test.clj     |     1 +
 .../test/clj/backtype/storm/supervisor_test.clj |    18 +-
 .../backtype/storm/blobstore/BlobStoreTest.java |   461 +
 .../storm/blobstore/BlobSynchronizerTest.java   |   137 +
 .../storm/blobstore/ClientBlobStoreTest.java    |   179 +
 .../LocalizedResourceRetentionSetTest.java      |    85 +
 .../localizer/LocalizedResourceSetTest.java     |    74 +
 .../backtype/storm/localizer/LocalizerTest.java |   671 +
 .../jvm/backtype/storm/localizer/localtest.zip  |   Bin 0 -> 6378 bytes
 .../storm/localizer/localtestwithsymlink.jar    |   Bin 0 -> 6591 bytes
 .../storm/localizer/localtestwithsymlink.tar    |   Bin 0 -> 24576 bytes
 .../storm/localizer/localtestwithsymlink.tar.gz |   Bin 0 -> 6106 bytes
 .../storm/localizer/localtestwithsymlink.tgz    |   Bin 0 -> 6106 bytes
 96 files changed, 39479 insertions(+), 9515 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/bin/storm.py
----------------------------------------------------------------------
diff --git a/bin/storm.py b/bin/storm.py
index 44d54d5..f943778 100755
--- a/bin/storm.py
+++ b/bin/storm.py
@@ -263,6 +263,32 @@ 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 ...] [--replication-factor 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.
+    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 --replication-factor NUMBER KEY where NUMBER > 0. It is used to update the
+        replication factor of a blob.
+    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
+    """
+    exec_storm_class(
+        "backtype.storm.command.blobstore",
+        args=args,
+        jvmtype="-client",
+        extrajars=[USER_CONF_DIR, STORM_BIN_DIR])
+
 def heartbeats(*args):
     """Syntax: [storm heartbeats [cmd]]
 
@@ -658,7 +684,7 @@ COMMANDS = {"jar": jar, "kill": kill, "shell": shell, "nimbus": nimbus, "ui": ui
             "remoteconfvalue": print_remoteconfvalue, "repl": repl, "classpath": print_classpath,
             "activate": activate, "deactivate": deactivate, "rebalance": rebalance, "help": print_usage,
             "list": listtopos, "dev-zookeeper": dev_zookeeper, "version": version, "monitor": monitor,
-            "upload-credentials": upload_credentials, "pacemaker": pacemaker, "heartbeats": heartbeats,
+            "upload-credentials": upload_credentials, "pacemaker": pacemaker, "heartbeats": heartbeats, "blobstore": blobstore,
             "get-errors": get_errors, "set_log_level": set_log_level, "kill_workers": kill_workers,
             "node-health-check": healthcheck}
 

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 295ac7c..aeda11c 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -66,7 +66,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
 nimbus.task.launch.secs: 120
 nimbus.file.copy.expiration.secs: 600
 nimbus.topology.validator: "backtype.storm.nimbus.DefaultTopologyValidator"
@@ -117,6 +117,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"
+storm.blobstore.replication.factor: 3
+
 ### supervisor.* configs are for node supervisors
 # Define the amount of workers that can be run on this machine. Each worker is assigned a port to use for communication
 supervisor.slots.ports:

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/external/storm-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml
index 1765be9..7fad1a3 100644
--- a/external/storm-hdfs/pom.xml
+++ b/external/storm-hdfs/pom.xml
@@ -93,6 +93,17 @@
         </dependency>
         <dependency>
             <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <version>${hadoop.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-minicluster</artifactId>
             <version>${hadoop.version}</version>
             <exclusions>
@@ -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>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-minikdc</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.directory.server</groupId>
+                    <artifactId>apacheds-kerberos-codec</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.httpcomponents</groupId>
+                    <artifactId>httpclient</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
             <groupId>org.mockito</groupId>
             <artifactId>mockito-all</artifactId>
             <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java
----------------------------------------------------------------------
diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java
new file mode 100644
index 0000000..144ad71
--- /dev/null
+++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java
@@ -0,0 +1,370 @@
+/**
+ * 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.
+            String principal = (String) conf.get(Config.BLOBSTORE_HDFS_PRINCIPAL);
+            String keyTab = (String) conf.get(Config.BLOBSTORE_HDFS_KEYTAB);
+
+            if (principal != null && keyTab != null) {
+                UserGroupInformation.loginUserFromKeytab(principal, keyTab);
+            } else {
+                if (principal == null && keyTab != null) {
+                    throw new RuntimeException("You must specify an HDFS principal to go with the keytab!");
+
+                } else {
+                    if (principal != null && keyTab == null) {
+                        throw new RuntimeException("You must specify HDFS keytab go with the principal!");
+                    }
+                }
+            }
+        } catch (IOException e) {
+            throw new RuntimeException("Error logging in from keytab!", e);
+        }
+        Path baseDir = new Path(overrideBase, BASE_BLOBS_DIR_NAME);
+        try {
+            if (hadoopConf != null) {
+                _hbs = new HdfsBlobStoreImpl(baseDir, conf, hadoopConf);
+            } else {
+                _hbs = new HdfsBlobStoreImpl(baseDir, conf);
+            }
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+        _localSubject = getHadoopUser();
+        _aclHandler = new BlobStoreAclHandler(conf);
+    }
+
+    @Override
+    public AtomicOutputStream createBlob(String key, SettableBlobMeta meta, Subject who)
+            throws AuthorizationException, KeyAlreadyExistsException {
+        if (meta.get_replication_factor() <= 0) {
+            meta.set_replication_factor((int)conf.get(Config.STORM_BLOBSTORE_REPLICATION_FACTOR));
+        }
+        who = checkAndGetSubject(who);
+        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 (_hbs.exists(DATA_PREFIX+key)) {
+            throw new KeyAlreadyExistsException(key);
+        }
+        BlobStoreFileOutputStream mOut = null;
+        try {
+            BlobStoreFile metaFile = _hbs.write(META_PREFIX + key, true);
+            metaFile.setMetadata(meta);
+            mOut = new BlobStoreFileOutputStream(metaFile);
+            mOut.write(Utils.thriftSerialize(meta));
+            mOut.close();
+            mOut = null;
+            BlobStoreFile dataFile = _hbs.write(DATA_PREFIX + key, true);
+            dataFile.setMetadata(meta);
+            return new BlobStoreFileOutputStream(dataFile);
+        } 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 {
+        who = checkAndGetSubject(who);
+        SettableBlobMeta meta = getStoredBlobMeta(key);
+        validateKey(key);
+        _aclHandler.hasPermissions(meta.get_acl(), WRITE, who, key);
+        try {
+            BlobStoreFile dataFile = _hbs.write(DATA_PREFIX + key, false);
+            dataFile.setMetadata(meta);
+            return new BlobStoreFileOutputStream(dataFile);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private SettableBlobMeta getStoredBlobMeta(String key) throws KeyNotFoundException {
+        InputStream in = null;
+        try {
+            BlobStoreFile pf = _hbs.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 {
+        who = checkAndGetSubject(who);
+        validateKey(key);
+        SettableBlobMeta meta = getStoredBlobMeta(key);
+        _aclHandler.validateUserCanReadMeta(meta.get_acl(), who, key);
+        ReadableBlobMeta rbm = new ReadableBlobMeta();
+        rbm.set_settable(meta);
+        try {
+            BlobStoreFile pf = _hbs.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 {
+        if (meta.get_replication_factor() <= 0) {
+            meta.set_replication_factor((int)conf.get(Config.STORM_BLOBSTORE_REPLICATION_FACTOR));
+        }
+        who = checkAndGetSubject(who);
+        validateKey(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;
+        writeMetadata(key, meta);
+    }
+
+    @Override
+    public void deleteBlob(String key, Subject who)
+            throws AuthorizationException, KeyNotFoundException {
+        who = checkAndGetSubject(who);
+        validateKey(key);
+        SettableBlobMeta meta = getStoredBlobMeta(key);
+        _aclHandler.hasPermissions(meta.get_acl(), WRITE, who, key);
+        try {
+            _hbs.deleteKey(DATA_PREFIX + key);
+            _hbs.deleteKey(META_PREFIX + key);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public InputStreamWithMeta getBlob(String key, Subject who)
+            throws AuthorizationException, KeyNotFoundException {
+        who = checkAndGetSubject(who);
+        validateKey(key);
+        SettableBlobMeta meta = getStoredBlobMeta(key);
+        _aclHandler.hasPermissions(meta.get_acl(), READ, who, key);
+        try {
+            return new BlobStoreFileInputStream(_hbs.read(DATA_PREFIX + key));
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public Iterator<String> listKeys() {
+        try {
+            return new KeyTranslationIterator(_hbs.listKeys(), DATA_PREFIX);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public void shutdown() {
+        //Empty
+    }
+
+    @Override
+    public int getBlobReplication(String key, Subject who) throws AuthorizationException, KeyNotFoundException {
+        who = checkAndGetSubject(who);
+        validateKey(key);
+        SettableBlobMeta meta = getStoredBlobMeta(key);
+        _aclHandler.hasAnyPermissions(meta.get_acl(), READ | WRITE | ADMIN, who, key);
+        try {
+            return _hbs.getBlobReplication(DATA_PREFIX + key);
+        } catch (IOException exp) {
+            throw new RuntimeException(exp);
+        }
+    }
+
+    @Override
+    public int updateBlobReplication(String key, int replication, Subject who) throws AuthorizationException, KeyNotFoundException {
+        who = checkAndGetSubject(who);
+        validateKey(key);
+        SettableBlobMeta meta = getStoredBlobMeta(key);
+        meta.set_replication_factor(replication);
+        _aclHandler.hasAnyPermissions(meta.get_acl(), WRITE | ADMIN, who, key);
+        try {
+            writeMetadata(key, meta);
+            return _hbs.updateBlobReplication(DATA_PREFIX + key, replication);
+        } catch (IOException exp) {
+            throw new RuntimeException(exp);
+        }
+    }
+
+    public void writeMetadata(String key, SettableBlobMeta meta)
+            throws AuthorizationException, KeyNotFoundException {
+        BlobStoreFileOutputStream mOut = null;
+        try {
+            BlobStoreFile hdfsFile = _hbs.write(META_PREFIX + key, false);
+            hdfsFile.setMetadata(meta);
+            mOut = new BlobStoreFileOutputStream(hdfsFile);
+            mOut.write(Utils.thriftSerialize(meta));
+            mOut.close();
+            mOut = null;
+        } catch (IOException exp) {
+            throw new RuntimeException(exp);
+        } finally {
+            if (mOut != null) {
+                try {
+                    mOut.cancel();
+                } catch (IOException e) {
+                    //Ignored
+                }
+            }
+        }
+    }
+
+    public void fullCleanup(long age) throws IOException {
+        _hbs.fullCleanup(age);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStoreFile.java
----------------------------------------------------------------------
diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStoreFile.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStoreFile.java
new file mode 100644
index 0000000..93b56c1
--- /dev/null
+++ b/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) {
+        if (BLOBSTORE_DATA_FILE.equals(name)) {
+            _isTmp = false;
+        } else {
+            Matcher m = TMP_NAME_PATTERN.matcher(name);
+            if (!m.matches()) {
+                throw new IllegalArgumentException("File name does not match '"+name+"' !~ "+TMP_NAME_PATTERN);
+            }
+            _isTmp = true;
+        }
+        _hadoopConf = hconf;
+        _key = base.getName();
+        _path = new Path(base, name);
+        _mustBeNew = false;
+        try {
+            _fs = _path.getFileSystem(_hadoopConf);
+        } catch (IOException e) {
+            throw new RuntimeException("Error getting filesystem for path: " + _path, e);
+        }
+    }
+
+    public HdfsBlobStoreFile(Path base, boolean isTmp, boolean mustBeNew, Configuration hconf) {
+        _key = base.getName();
+        _hadoopConf = hconf;
+        _isTmp = isTmp;
+        _mustBeNew = mustBeNew;
+        if (_isTmp) {
+            _path = new Path(base, System.currentTimeMillis()+TMP_EXT);
+        } else {
+            _path = new Path(base, BLOBSTORE_DATA_FILE);
+        }
+        try {
+            _fs = _path.getFileSystem(_hadoopConf);
+        } catch (IOException e) {
+            throw new RuntimeException("Error getting filesystem for path: " + _path, e);
+        }
+    }
+
+    @Override
+    public void delete() throws IOException {
+        _fs.delete(_path, true);
+    }
+
+    @Override
+    public boolean isTmp() {
+        return _isTmp;
+    }
+
+    @Override
+    public String getKey() {
+        return _key;
+    }
+
+    @Override
+    public long getModTime() throws IOException {
+        if (_modTime == null) {
+            FileSystem fs = _path.getFileSystem(_hadoopConf);
+            _modTime = fs.getFileStatus(_path).getModificationTime();
+        }
+        return _modTime;
+    }
+
+    private void checkIsNotTmp() {
+        if (!isTmp()) {
+            throw new IllegalStateException("Can only operate on a temporary blobstore file.");
+        }
+    }
+
+    private void checkIsTmp() {
+        if (isTmp()) {
+            throw new IllegalStateException("Cannot operate on a temporary blobstore file.");
+        }
+    }
+
+    @Override
+    public InputStream getInputStream() throws IOException {
+        checkIsTmp();
+        return _fs.open(_path);
+    }
+
+    @Override
+    public OutputStream getOutputStream() throws IOException {
+        checkIsNotTmp();
+        OutputStream out = null;
+        FsPermission fileperms = new FsPermission(BLOBSTORE_FILE_PERMISSION);
+        try {
+            out = _fs.create(_path, (short)this.getMetadata().get_replication_factor());
+            _fs.setPermission(_path, fileperms);
+            _fs.setReplication(_path, (short)this.getMetadata().get_replication_factor());
+        } catch (IOException e) {
+            //Try to create the parent directory, may not work
+            FsPermission dirperms = new FsPermission(HdfsBlobStoreImpl.BLOBSTORE_DIR_PERMISSION);
+            if (!_fs.mkdirs(_path.getParent(), dirperms)) {
+                LOG.warn("error creating parent dir: " + _path.getParent());
+            }
+            out = _fs.create(_path, (short)this.getMetadata().get_replication_factor());
+            _fs.setPermission(_path, dirperms);
+            _fs.setReplication(_path, (short)this.getMetadata().get_replication_factor());
+        }
+        if (out == null) {
+            throw new IOException("Error in creating: " + _path);
+        }
+        return out;
+    }
+
+    @Override
+    public void commit() throws IOException {
+        checkIsNotTmp();
+        // FileContext supports atomic rename, whereas FileSystem doesn't
+        FileContext fc = FileContext.getFileContext(_hadoopConf);
+        Path dest = new Path(_path.getParent(), BLOBSTORE_DATA_FILE);
+        if (_mustBeNew) {
+            fc.rename(_path, dest);
+        } else {
+            fc.rename(_path, dest, Options.Rename.OVERWRITE);
+        }
+        // Note, we could add support for setting the replication factor
+    }
+
+    @Override
+    public void cancel() throws IOException {
+        checkIsNotTmp();
+        delete();
+    }
+
+    @Override
+    public String toString() {
+        return _path+":"+(_isTmp ? "tmp": BlobStoreFile.BLOBSTORE_DATA_FILE)+":"+_key;
+    }
+
+    @Override
+    public long getFileLength() throws IOException {
+        return _fs.getFileStatus(_path).getLen();
+    }
+
+    @Override
+    public SettableBlobMeta getMetadata() {
+        return meta;
+    }
+
+    @Override
+    public void setMetadata(SettableBlobMeta meta) {
+        this.meta = meta;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStoreImpl.java
----------------------------------------------------------------------
diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStoreImpl.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStoreImpl.java
new file mode 100644
index 0000000..e434752
--- /dev/null
+++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStoreImpl.java
@@ -0,0 +1,312 @@
+/**
+ * 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.BlobStoreFile;
+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.ArrayList;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Timer;
+import java.util.TimerTask;
+
+/**
+ * HDFS blob store impl.
+ */
+public class HdfsBlobStoreImpl {
+    private static final Logger LOG = LoggerFactory.getLogger(HdfsBlobStoreImpl.class);
+
+    private static final long FULL_CLEANUP_FREQ = 60 * 60 * 1000l;
+    private static final int BUCKETS = 1024;
+    private static final Timer timer = new Timer("HdfsBlobStore cleanup thread", true);
+    private static final String BLOBSTORE_DATA = "data";
+
+    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);
+                Path dir = new Path(_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 Path _fullPath;
+    private FileSystem _fs;
+    private TimerTask _cleanup = null;
+    private Configuration _hadoopConf;
+
+    // blobstore directory is private!
+    final public static FsPermission BLOBSTORE_DIR_PERMISSION =
+            FsPermission.createImmutable((short) 0700); // rwx--------
+
+    public HdfsBlobStoreImpl(Path path, Map<String, Object> conf) throws IOException {
+        this(path, conf, new Configuration());
+    }
+
+    public HdfsBlobStoreImpl(Path path, Map<String, Object> conf,
+                             Configuration hconf) throws IOException {
+        LOG.info("Blob store based in {}", path);
+        _fullPath = path;
+        _hadoopConf = hconf;
+        _fs = path.getFileSystem(_hadoopConf);
+
+        if (!_fs.exists(_fullPath)) {
+            FsPermission perms = new FsPermission(BLOBSTORE_DIR_PERMISSION);
+            boolean success = _fs.mkdirs(_fullPath, perms);
+            if (!success) {
+                throw new IOException("Error creating blobstore directory: " + _fullPath);
+            }
+        }
+
+        Object shouldCleanup = conf.get(Config.BLOBSTORE_CLEANUP_ENABLE);
+        if (Utils.getBoolean(shouldCleanup, false)) {
+            LOG.debug("Starting hdfs 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 BlobStoreFile read(String key) throws IOException {
+        return new HdfsBlobStoreFile(getKeyDir(key), BLOBSTORE_DATA, _hadoopConf);
+    }
+
+    /**
+     * Get an object tied to writing the data.
+     *
+     * @param key the key of the part to write to.
+     * @param create whether the file needs to be new or not.
+     * @return an object that can be used to both write to, but also commit/cancel the operation.
+     * @throws IOException on any error
+     */
+    public BlobStoreFile write(String key, boolean create) throws IOException {
+        return new HdfsBlobStoreFile(getKeyDir(key), true, create, _hadoopConf);
+    }
+
+    /**
+     * 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) {
+        Path dir = getKeyDir(key);
+        boolean res = false;
+        try {
+            _fs = dir.getFileSystem(_hadoopConf);
+            res = _fs.exists(dir);
+        } catch (IOException e) {
+            LOG.warn("Exception checking for exists on: " + key);
+        }
+        return res;
+    }
+
+    /**
+     * 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 {
+        Path keyDir = getKeyDir(key);
+        HdfsBlobStoreFile pf = new HdfsBlobStoreFile(keyDir, BLOBSTORE_DATA,
+                _hadoopConf);
+        pf.delete();
+        delete(keyDir);
+    }
+
+    protected Path getKeyDir(String key) {
+        String hash = String.valueOf(Math.abs((long) key.hashCode()) % BUCKETS);
+        Path hashDir = new Path(_fullPath, hash);
+
+        Path ret = new Path(hashDir, 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();
+            Path keyDir = getKeyDir(key);
+            Iterator<BlobStoreFile> i = listBlobStoreFiles(keyDir);
+            if (!i.hasNext()) {
+                //The dir is empty, so try to delete it, may fail, but that is OK
+                try {
+                    _fs.delete(keyDir, true);
+                } catch (Exception e) {
+                    LOG.warn("Could not delete " + keyDir + " will try again later");
+                }
+            }
+            while (i.hasNext()) {
+                BlobStoreFile f = i.next();
+                if (f.isTmp()) {
+                    if (f.getModTime() <= cleanUpIfBefore) {
+                        f.delete();
+                    }
+                }
+            }
+        }
+    }
+
+    protected Iterator<BlobStoreFile> listBlobStoreFiles(Path path) throws IOException {
+        ArrayList<BlobStoreFile> ret = new ArrayList<BlobStoreFile>();
+        FileStatus[] files = _fs.listStatus(new Path[]{path});
+        if (files != null) {
+            for (FileStatus sub : files) {
+                try {
+                    ret.add(new HdfsBlobStoreFile(sub.getPath().getParent(), sub.getPath().getName(),
+                            _hadoopConf));
+                } catch (IllegalArgumentException e) {
+                    //Ignored the file did not match
+                    LOG.warn("Found an unexpected file in {} {}", path, sub.getPath().getName());
+                }
+            }
+        }
+        return ret.iterator();
+    }
+
+    protected Iterator<String> listKeys(Path path) throws IOException {
+        ArrayList<String> ret = new ArrayList<String>();
+        FileStatus[] files = _fs.listStatus(new Path[]{path});
+        if (files != null) {
+            for (FileStatus sub : files) {
+                try {
+                    ret.add(sub.getPath().getName().toString());
+                } catch (IllegalArgumentException e) {
+                    //Ignored the file did not match
+                    LOG.debug("Found an unexpected file in {} {}", path, sub.getPath().getName());
+                }
+            }
+        }
+        return ret.iterator();
+    }
+
+    protected int getBlobReplication(String key) throws IOException {
+        Path path = getKeyDir(key);
+        Path dest = new Path(path, BLOBSTORE_DATA);
+        return _fs.getFileStatus(dest).getReplication();
+    }
+
+    protected int updateBlobReplication(String key, int replication) throws IOException {
+        Path path = getKeyDir(key);
+        Path dest = new Path(path, BLOBSTORE_DATA);
+        _fs.setReplication(dest, (short) replication);
+        return _fs.getFileStatus(dest).getReplication();
+    }
+
+    protected void delete(Path path) throws IOException {
+        _fs.delete(path, true);
+    }
+
+    public void shutdown() {
+        if (_cleanup != null) {
+            _cleanup.cancel();
+            _cleanup = null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsClientBlobStore.java
----------------------------------------------------------------------
diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsClientBlobStore.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsClientBlobStore.java
new file mode 100644
index 0000000..ec17dae
--- /dev/null
+++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsClientBlobStore.java
@@ -0,0 +1,115 @@
+/**
+ * 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.AtomicOutputStream;
+import backtype.storm.blobstore.ClientBlobStore;
+import backtype.storm.blobstore.InputStreamWithMeta;
+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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Iterator;
+import java.util.Map;
+
+/**
+ *  Client to access the HDFS blobStore. At this point, this is meant to only be used by the
+ *  supervisor.  Don't trust who the client says they are so pass null for all Subjects.
+ */
+public class HdfsClientBlobStore extends ClientBlobStore {
+    private static final Logger LOG = LoggerFactory.getLogger(HdfsClientBlobStore.class);
+    private HdfsBlobStore _blobStore;
+    private Map _conf;
+
+    @Override
+    public void prepare(Map conf) {
+        this._conf = conf;
+        _blobStore = new HdfsBlobStore();
+        _blobStore.prepare(conf, null, null);
+    }
+
+    @Override
+    public AtomicOutputStream createBlobToExtend(String key, SettableBlobMeta meta)
+            throws AuthorizationException, KeyAlreadyExistsException {
+        return _blobStore.createBlob(key, meta, null);
+    }
+
+    @Override
+    public AtomicOutputStream updateBlob(String key)
+            throws AuthorizationException, KeyNotFoundException {
+        return _blobStore.updateBlob(key, null);
+    }
+
+    @Override
+    public ReadableBlobMeta getBlobMeta(String key)
+            throws AuthorizationException, KeyNotFoundException {
+        return _blobStore.getBlobMeta(key, null);
+    }
+
+    @Override
+    public void setBlobMetaToExtend(String key, SettableBlobMeta meta)
+            throws AuthorizationException, KeyNotFoundException {
+        _blobStore.setBlobMeta(key, meta, null);
+    }
+
+    @Override
+    public void deleteBlob(String key) throws AuthorizationException, KeyNotFoundException {
+        _blobStore.deleteBlob(key, null);
+    }
+
+    @Override
+    public InputStreamWithMeta getBlob(String key)
+            throws AuthorizationException, KeyNotFoundException {
+        return _blobStore.getBlob(key, null);
+    }
+
+    @Override
+    public Iterator<String> listKeys() {
+        return _blobStore.listKeys();
+    }
+
+    @Override
+    public int getBlobReplication(String key) throws AuthorizationException, KeyNotFoundException {
+        return _blobStore.getBlobReplication(key, null);
+    }
+
+    @Override
+    public int updateBlobReplication(String key, int replication) throws AuthorizationException, KeyNotFoundException {
+        return _blobStore.updateBlobReplication(key, replication, null);
+    }
+
+    @Override
+    public boolean setClient(Map conf, NimbusClient client) {
+        return true;
+    }
+
+    @Override
+    public void createStateInZookeeper(String key) {
+        // Do nothing
+    }
+
+    @Override
+    public void shutdown() {
+        // do nothing
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/ha/codedistributor/HDFSCodeDistributor.java
----------------------------------------------------------------------
diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/ha/codedistributor/HDFSCodeDistributor.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/ha/codedistributor/HDFSCodeDistributor.java
deleted file mode 100644
index 1e38051..0000000
--- a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/ha/codedistributor/HDFSCodeDistributor.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/**
- * 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.ha.codedistributor;
-
-import backtype.storm.codedistributor.ICodeDistributor;
-import com.google.common.collect.Lists;
-import org.apache.commons.io.IOUtils;
-import org.apache.commons.lang.Validate;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.storm.hdfs.common.security.HdfsSecurityUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.*;
-import java.util.List;
-import java.util.Map;
-
-public class HDFSCodeDistributor implements ICodeDistributor {
-    private static final Logger LOG = LoggerFactory.getLogger(HDFSCodeDistributor.class);
-
-    private final static String HDFS_STORM_DIR = "hdfs.storm.dir";
-
-    private FileSystem fs;
-    private Path stormDir;
-
-    @Override
-    public void prepare(Map conf) throws Exception {
-        Validate.notNull(conf.get(HDFS_STORM_DIR), "you must specify " + HDFS_STORM_DIR);
-
-        Configuration configuration = new Configuration();
-        HdfsSecurityUtil.login(conf, configuration);
-        this.fs = FileSystem.get(configuration);
-        this.stormDir = new Path(String.valueOf(conf.get(HDFS_STORM_DIR)));
-        if(!this.fs.exists(stormDir)) {
-            this.fs.mkdirs(this.stormDir);
-        }
-    }
-
-    @Override
-    public File upload(String dirPath, String topologyId) throws Exception {
-        File localStormDir = new File(dirPath);
-        LOG.info("Copying the storm code from directory: {} to {}{}{}", localStormDir.getAbsolutePath(),
-                stormDir.toString(), Path.SEPARATOR , topologyId);
-
-        File[] files = localStormDir.listFiles();
-
-        Path hdfsDestPath = new Path(stormDir, new Path(topologyId));
-        fs.mkdirs(hdfsDestPath);
-
-        for(File file : files) {
-            fs.copyFromLocalFile(new Path(file.getAbsolutePath()), hdfsDestPath);
-        }
-
-        File file = new File(dirPath, "storm-code-distributor.meta");
-
-        RemoteIterator<LocatedFileStatus> hdfsFileIterator = fs.listFiles(hdfsDestPath, false);
-
-        BufferedWriter writer = new BufferedWriter(new FileWriter(file));
-        while(hdfsFileIterator.hasNext()) {
-            writer.write(hdfsFileIterator.next().getPath().toString());
-            writer.newLine();
-        }
-        writer.close();
-
-        return file;
-    }
-
-    @Override
-    public List<File> download(String topologyId, File metaFile) throws Exception {
-        File destDir = metaFile.getParentFile();
-
-        List<String> hdfsPaths = IOUtils.readLines(new FileInputStream(metaFile));
-        for(String hdfsFilePath : hdfsPaths) {
-            fs.copyToLocalFile(new Path(hdfsFilePath), new Path(destDir.getAbsolutePath()));
-        }
-
-        return Lists.newArrayList(destDir.listFiles());
-    }
-
-    @Override
-    public short getReplicationCount(String topologyId) throws IOException {
-        Path hdfsDestPath = new Path(stormDir, new Path(topologyId));
-        if(fs.exists(hdfsDestPath)) {
-            FileStatus fileStatus = fs.getFileStatus(hdfsDestPath);
-            return fileStatus.getReplication();
-        } else {
-            LOG.warn("getReplicationCount called for {} but no such directory exists, returning 0", topologyId);
-            return 0;
-        }
-    }
-
-    @Override
-    public void cleanup(String topologyId) throws IOException {
-        Path hdfsDestPath = new Path(stormDir, new Path(topologyId));
-        fs.delete(hdfsDestPath, true);
-    }
-
-    @Override
-    public void close(Map conf) {
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/blobstore/BlobStoreTest.java
----------------------------------------------------------------------
diff --git a/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/blobstore/BlobStoreTest.java b/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/blobstore/BlobStoreTest.java
new file mode 100644
index 0000000..a8d6172
--- /dev/null
+++ b/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/blobstore/BlobStoreTest.java
@@ -0,0 +1,530 @@
+/**
+ * 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 {
+    InputStream in = store.getBlob(key, who);
+    try {
+      return in.read();
+    } finally {
+      in.close();
+    }
+  }
+
+  public static int readInt(BlobStore store, String key)
+          throws IOException, KeyNotFoundException, AuthorizationException {
+    return readInt(store, null, key);
+  }
+
+  public static void readAssertEquals(BlobStore store, String key, int value)
+          throws IOException, KeyNotFoundException, AuthorizationException {
+    assertEquals(value, readInt(store, key));
+  }
+
+  // Checks for assertion when we turn on security
+  public void readAssertEqualsWithAuth(BlobStore store, Subject who, String key, int value)
+          throws IOException, KeyNotFoundException, AuthorizationException {
+    assertEquals(value, readInt(store, who, key));
+  }
+
+  private HdfsBlobStore initHdfs(String dirName)
+          throws Exception {
+    if (hadoopConf == null) {
+      hadoopConf = new Configuration();
+    }
+    try {
+      if (dfscluster == null) {
+        dfscluster = new MiniDFSCluster.Builder(hadoopConf).numDataNodes(3).build();
+        dfscluster.waitActive();
+      }
+    } catch (IOException e) {
+      LOG.error("error creating MiniDFSCluster");
+    }
+    Map conf = new HashMap();
+    conf.put(Config.BLOBSTORE_DIR, dirName);
+    conf.put(Config.STORM_PRINCIPAL_TO_LOCAL_PLUGIN,"backtype.storm.security.auth.DefaultPrincipalToLocal");
+    conf.put(Config.STORM_BLOBSTORE_REPLICATION_FACTOR, 3);
+    HdfsBlobStore store = new HdfsBlobStore();
+    store.prepareInternal(conf, null, dfscluster.getConfiguration(0));
+    return store;
+  }
+
+  @Test
+  public void testHdfsReplication()
+          throws Exception {
+    BlobStore store = initHdfs("/storm/blobstoreReplication");
+    testReplication("/storm/blobstoreReplication/test", store);
+  }
+
+  @Test
+  public void testBasicHdfs()
+          throws Exception {
+    testBasic(initHdfs("/storm/blobstore1"));
+  }
+
+  @Test
+  public void testMultipleHdfs()
+          throws Exception {
+    // use different blobstore dir so it doesn't conflict with other test
+    testMultiple(initHdfs("/storm/blobstore2"));
+  }
+
+  @Test
+  public void testHdfsWithAuth()
+          throws Exception {
+    // use different blobstore dir so it doesn't conflict with other tests
+    testWithAuthentication(initHdfs("/storm/blobstore3"));
+  }
+
+  // Test for replication.
+  public void testReplication(String path, BlobStore store)
+          throws Exception {
+    SettableBlobMeta metadata = new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING);
+    metadata.set_replication_factor(4);
+    AtomicOutputStream out = store.createBlob("test", metadata, null);
+    out.write(1);
+    out.close();
+    assertStoreHasExactly(store, "test");
+    assertEquals("Blobstore replication not matching", store.getBlobReplication("test", null), 4);
+    store.deleteBlob("test", null);
+
+    //Test for replication with NIMBUS as user
+    Subject admin = getSubject("admin");
+    metadata = new SettableBlobMeta(BlobStoreAclHandler.DEFAULT);
+    metadata.set_replication_factor(4);
+    out = store.createBlob("test", metadata, admin);
+    out.write(1);
+    out.close();
+    assertStoreHasExactly(store, "test");
+    assertEquals("Blobstore replication not matching", store.getBlobReplication("test", admin), 4);
+    store.updateBlobReplication("test", 5, admin);
+    assertEquals("Blobstore replication not matching", store.getBlobReplication("test", admin), 5);
+    store.deleteBlob("test", admin);
+
+    //Test for replication using SUPERVISOR access
+    Subject supervisor = getSubject("supervisor");
+    metadata = new SettableBlobMeta(BlobStoreAclHandler.DEFAULT);
+    metadata.set_replication_factor(4);
+    out = store.createBlob("test", metadata, supervisor);
+    out.write(1);
+    out.close();
+    assertStoreHasExactly(store, "test");
+    assertEquals("Blobstore replication not matching", store.getBlobReplication("test", supervisor), 4);
+    store.updateBlobReplication("test", 5, supervisor);
+    assertEquals("Blobstore replication not matching", store.getBlobReplication("test", supervisor), 5);
+    store.deleteBlob("test", supervisor);
+
+    //Test for a user having read or write or admin access to read replication for a blob
+    String createSubject = "createSubject";
+    String writeSubject = "writeSubject";
+    String adminSubject = "adminSubject";
+    Subject who = getSubject(createSubject);
+    AccessControl writeAccess = new AccessControl(AccessControlType.USER, READ);
+    AccessControl adminAccess = new AccessControl(AccessControlType.USER, ADMIN);
+    writeAccess.set_name(writeSubject);
+    adminAccess.set_name(adminSubject);
+    List<AccessControl> acl = Arrays.asList(writeAccess, adminAccess);
+    metadata = new SettableBlobMeta(acl);
+    metadata.set_replication_factor(4);
+    out = store.createBlob("test", metadata, who);
+    out.write(1);
+    out.close();
+    assertStoreHasExactly(store, "test");
+    who = getSubject(writeSubject);
+    assertEquals("Blobstore replication not matching", store.getBlobReplication("test", who), 4);
+
+    //Test for a user having WRITE or ADMIN privileges to change replication of a blob
+    who = getSubject(adminSubject);
+    store.updateBlobReplication("test", 5, who);
+    assertEquals("Blobstore replication not matching", store.getBlobReplication("test", who), 5);
+    store.deleteBlob("test", getSubject(createSubject));
+  }
+
+  public Subject getSubject(String name) {
+    Subject subject = new Subject();
+    SingleUserPrincipal user = new SingleUserPrincipal(name);
+    subject.getPrincipals().add(user);
+    return subject;
+  }
+
+  // Check for Blobstore with authentication
+  public void testWithAuthentication(BlobStore store)
+          throws Exception {
+    //Test for Nimbus Admin
+    Subject admin = getSubject("admin");
+    assertStoreHasExactly(store);
+    SettableBlobMeta metadata = new SettableBlobMeta(BlobStoreAclHandler.DEFAULT);
+    AtomicOutputStream out = store.createBlob("test", metadata, admin);
+    assertStoreHasExactly(store, "test");
+    out.write(1);
+    out.close();
+    store.deleteBlob("test", admin);
+
+    //Test for Supervisor Admin
+    Subject supervisor = getSubject("supervisor");
+    assertStoreHasExactly(store);
+    metadata = new SettableBlobMeta(BlobStoreAclHandler.DEFAULT);
+    out = store.createBlob("test", metadata, supervisor);
+    assertStoreHasExactly(store, "test");
+    out.write(1);
+    out.close();
+    store.deleteBlob("test", supervisor);
+
+    //Test for Nimbus itself as a user
+    Subject nimbus = getNimbusSubject();
+    assertStoreHasExactly(store);
+    metadata = new SettableBlobMeta(BlobStoreAclHandler.DEFAULT);
+    out = store.createBlob("test", metadata, nimbus);
+    assertStoreHasExactly(store, "test");
+    out.write(1);
+    out.close();
+    store.deleteBlob("test", nimbus);
+
+    // Test with a dummy test_subject for cases where subject !=null (security turned on)
+    Subject who = getSubject("test_subject");
+    assertStoreHasExactly(store);
+
+    // Tests for case when subject != null (security turned on) and
+    // acls for the blob are set to WORLD_EVERYTHING
+    metadata = new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING);
+    out = store.createBlob("test", metadata, who);
+    out.write(1);
+    out.close();
+    assertStoreHasExactly(store, "test");
+    // Testing whether acls are set to WORLD_EVERYTHING
+    assertTrue("ACL does not contain WORLD_EVERYTHING", metadata.toString().contains("AccessControl(type:OTHER, access:7)"));
+    readAssertEqualsWithAuth(store, who, "test", 1);
+
+    LOG.info("Deleting test");
+    store.deleteBlob("test", who);
+    assertStoreHasExactly(store);
+
+    // Tests for case when subject != null (security turned on) and
+    // acls are not set for the blob (DEFAULT)
+    LOG.info("Creating test again");
+    metadata = new SettableBlobMeta(BlobStoreAclHandler.DEFAULT);
+    out = store.createBlob("test", metadata, who);
+    out.write(2);
+    out.close();
+    assertStoreHasExactly(store, "test");
+    // Testing whether acls are set to WORLD_EVERYTHING. Here the acl should not contain WORLD_EVERYTHING because
+    // the subject is neither null nor empty. The ACL should however contain USER_EVERYTHING as user needs to have
+    // complete access to the blob
+    assertTrue("ACL does not contain WORLD_EVERYTHING", !metadata.toString().contains("AccessControl(type:OTHER, access:7)"));
+    readAssertEqualsWithAuth(store, who, "test", 2);
+
+    LOG.info("Updating test");
+    out = store.updateBlob("test", who);
+    out.write(3);
+    out.close();
+    assertStoreHasExactly(store, "test");
+    readAssertEqualsWithAuth(store, who, "test", 3);
+
+    LOG.info("Updating test again");
+    out = store.updateBlob("test", who);
+    out.write(4);
+    out.flush();
+    LOG.info("SLEEPING");
+    Thread.sleep(2);
+    assertStoreHasExactly(store, "test");
+    readAssertEqualsWithAuth(store, who, "test", 3);
+
+    //Test for subject with no principals and acls set to WORLD_EVERYTHING
+    who = new Subject();
+    metadata = new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING);
+    LOG.info("Creating test");
+    out = store.createBlob("test-empty-subject-WE", metadata, who);
+    out.write(2);
+    out.close();
+    assertStoreHasExactly(store, "test-empty-subject-WE", "test");
+    // Testing whether acls are set to WORLD_EVERYTHING
+    assertTrue("ACL does not contain WORLD_EVERYTHING", metadata.toString().contains("AccessControl(type:OTHER, access:7)"));
+    readAssertEqualsWithAuth(store, who, "test-empty-subject-WE", 2);
+
+    //Test for subject with no principals and acls set to DEFAULT
+    who = new Subject();
+    metadata = new SettableBlobMeta(BlobStoreAclHandler.DEFAULT);
+    LOG.info("Creating other");
+    out = store.createBlob("test-empty-subject-DEF", metadata, who);
+    out.write(2);
+    out.close();
+    assertStoreHasExactly(store, "test-empty-subject-DEF", "test", "test-empty-subject-WE");
+    // Testing whether acls are set to WORLD_EVERYTHING
+    assertTrue("ACL does not contain WORLD_EVERYTHING", metadata.toString().contains("AccessControl(type:OTHER, access:7)"));
+    readAssertEqualsWithAuth(store, who, "test-empty-subject-DEF", 2);
+
+    if (store instanceof HdfsBlobStore) {
+      ((HdfsBlobStore) store).fullCleanup(1);
+    } else {
+      fail("Error the blobstore is of unknowntype");
+    }
+    try {
+      out.close();
+    } catch (IOException e) {
+      //This is likely to happen when we try to commit something that
+      // was cleaned up.  This is expected and acceptable.
+    }
+  }
+
+  public void testBasic(BlobStore store)
+          throws Exception {
+    assertStoreHasExactly(store);
+    LOG.info("Creating test");
+    // Tests for case when subject == null (security turned off) and
+    // acls for the blob are set to WORLD_EVERYTHING
+    SettableBlobMeta metadata = new SettableBlobMeta(BlobStoreAclHandler
+            .WORLD_EVERYTHING);
+    AtomicOutputStream out = store.createBlob("test", metadata, null);
+    out.write(1);
+    out.close();
+    assertStoreHasExactly(store, "test");
+    // Testing whether acls are set to WORLD_EVERYTHING
+    assertTrue("ACL does not contain WORLD_EVERYTHING", metadata.toString().contains("AccessControl(type:OTHER, access:7)"));
+    readAssertEquals(store, "test", 1);
+
+    LOG.info("Deleting test");
+    store.deleteBlob("test", null);
+    assertStoreHasExactly(store);
+
+    // The following tests are run for both hdfs and local store to test the
+    // update blob interface
+    metadata = new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING);
+    LOG.info("Creating test again");
+    out = store.createBlob("test", metadata, null);
+    out.write(2);
+    out.close();
+    assertStoreHasExactly(store, "test");
+    readAssertEquals(store, "test", 2);
+    LOG.info("Updating test");
+    out = store.updateBlob("test", null);
+    out.write(3);
+    out.close();
+    assertStoreHasExactly(store, "test");
+    readAssertEquals(store, "test", 3);
+
+    LOG.info("Updating test again");
+    out = store.updateBlob("test", null);
+    out.write(4);
+    out.flush();
+    LOG.info("SLEEPING");
+    Thread.sleep(2);
+
+    if (store instanceof HdfsBlobStore) {
+      ((HdfsBlobStore) store).fullCleanup(1);
+    } else {
+      fail("Error the blobstore is of unknowntype");
+    }
+    try {
+      out.close();
+    } catch (IOException e) {
+      //This is likely to happen when we try to commit something that
+      // was cleaned up.  This is expected and acceptable.
+    }
+  }
+
+
+  public void testMultiple(BlobStore store)
+          throws Exception {
+    assertStoreHasExactly(store);
+    LOG.info("Creating test");
+    AtomicOutputStream out = store.createBlob("test", new SettableBlobMeta(BlobStoreAclHandler
+            .WORLD_EVERYTHING), null);
+    out.write(1);
+    out.close();
+    assertStoreHasExactly(store, "test");
+    readAssertEquals(store, "test", 1);
+
+    LOG.info("Creating other");
+    out = store.createBlob("other", new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING),
+            null);
+    out.write(2);
+    out.close();
+    assertStoreHasExactly(store, "test", "other");
+    readAssertEquals(store, "test", 1);
+    readAssertEquals(store, "other", 2);
+
+    LOG.info("Updating other");
+    out = store.updateBlob("other", null);
+    out.write(5);
+    out.close();
+    assertStoreHasExactly(store, "test", "other");
+    readAssertEquals(store, "test", 1);
+    readAssertEquals(store, "other", 5);
+
+    LOG.info("Deleting test");
+    store.deleteBlob("test", null);
+    assertStoreHasExactly(store, "other");
+    readAssertEquals(store, "other", 5);
+
+    LOG.info("Creating test again");
+    out = store.createBlob("test", new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING),
+            null);
+    out.write(2);
+    out.close();
+    assertStoreHasExactly(store, "test", "other");
+    readAssertEquals(store, "test", 2);
+    readAssertEquals(store, "other", 5);
+
+    LOG.info("Updating test");
+    out = store.updateBlob("test", null);
+    out.write(3);
+    out.close();
+    assertStoreHasExactly(store, "test", "other");
+    readAssertEquals(store, "test", 3);
+    readAssertEquals(store, "other", 5);
+
+    LOG.info("Deleting other");
+    store.deleteBlob("other", null);
+    assertStoreHasExactly(store, "test");
+    readAssertEquals(store, "test", 3);
+
+    LOG.info("Updating test again");
+    out = store.updateBlob("test", null);
+    out.write(4);
+    out.flush();
+    LOG.info("SLEEPING");
+    Thread.sleep(2);
+
+    if (store instanceof HdfsBlobStore) {
+      ((HdfsBlobStore) store).fullCleanup(1);
+    } else {
+      fail("Error the blobstore is of unknowntype");
+    }    assertStoreHasExactly(store, "test");
+    readAssertEquals(store, "test", 3);
+    try {
+      out.close();
+    } catch (IOException e) {
+      //This is likely to happen when we try to commit something that
+      // was cleaned up.  This is expected and acceptable.
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/blobstore/HdfsBlobStoreImplTest.java
----------------------------------------------------------------------
diff --git a/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/blobstore/HdfsBlobStoreImplTest.java b/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/blobstore/HdfsBlobStoreImplTest.java
new file mode 100644
index 0000000..cf2f7c1
--- /dev/null
+++ b/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/blobstore/HdfsBlobStoreImplTest.java
@@ -0,0 +1,231 @@
+/**
+ * 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.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.Charset;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import static org.junit.Assert.*;
+
+public class HdfsBlobStoreImplTest {
+    private static final Logger LOG = LoggerFactory.getLogger(HdfsBlobStoreImplTest.class);
+
+    protected static Configuration hadoopConf;
+    protected static MiniDFSCluster dfscluster;
+    // key dir needs to be number 0 to number of buckets, choose one so we know where to look
+    private static String KEYDIR = "0";
+    private Path blobDir = new Path("/storm/blobstore1");
+    private Path fullKeyDir = new Path(blobDir, KEYDIR);
+    private String BLOBSTORE_DATA = "data";
+
+    public class TestHdfsBlobStoreImpl extends HdfsBlobStoreImpl {
+
+        public TestHdfsBlobStoreImpl(Path path, Map<String, Object> conf) throws IOException {
+            super(path, conf);
+        }
+
+        public TestHdfsBlobStoreImpl(Path path, Map<String, Object> conf,
+                                     Configuration hconf) throws IOException {
+            super(path, conf, hconf);
+        }
+
+        protected Path getKeyDir(String key) {
+            return new Path(new Path(blobDir, KEYDIR), key);
+        }
+    }
+
+    @BeforeClass
+    public static void init() {
+        if (hadoopConf == null) {
+            hadoopConf = new Configuration();
+        }
+        try {
+            if (dfscluster == null) {
+                dfscluster = new MiniDFSCluster.Builder(hadoopConf).build();
+                dfscluster.waitActive();
+            }
+        } catch (IOException e) {
+            LOG.error("error creating MiniDFSCluster");
+        }
+    }
+
+    @AfterClass
+    public static void cleanup() throws IOException {
+        if (dfscluster != null) {
+            dfscluster.shutdown();
+        }
+    }
+
+    // Be careful about adding additional tests as the dfscluster will be shared
+
+    @Test
+    public void testMultiple() throws Exception {
+        String testString = "testingblob";
+        String validKey = "validkeyBasic";
+
+        FileSystem fs = dfscluster.getFileSystem();
+        Map conf = new HashMap();
+
+        TestHdfsBlobStoreImpl hbs = new TestHdfsBlobStoreImpl(blobDir, conf, hadoopConf);
+        // should have created blobDir
+        assertTrue("BlobStore dir wasn't created", fs.exists(blobDir));
+        assertEquals("BlobStore dir was created with wrong permissions",
+                HdfsBlobStoreImpl.BLOBSTORE_DIR_PERMISSION, fs.getFileStatus(blobDir).getPermission());
+
+        // test exist with non-existent key
+        assertFalse("file exists but shouldn't", hbs.exists("bogus"));
+
+        // test write
+        BlobStoreFile pfile = hbs.write(validKey, false);
+        // Adding metadata to avoid null pointer exception
+        SettableBlobMeta meta = new SettableBlobMeta();
+        meta.set_replication_factor(1);
+        pfile.setMetadata(meta);
+        OutputStream ios = pfile.getOutputStream();
+        ios.write(testString.getBytes(Charset.forName("UTF-8")));
+        ios.close();
+
+        // test commit creates properly
+        assertTrue("BlobStore key dir wasn't created", fs.exists(fullKeyDir));
+        pfile.commit();
+        Path dataFile = new Path(new Path(fullKeyDir, validKey), BLOBSTORE_DATA);
+        assertTrue("blob data not committed", fs.exists(dataFile));
+        assertEquals("BlobStore dir was created with wrong permissions",
+                HdfsBlobStoreFile.BLOBSTORE_FILE_PERMISSION, fs.getFileStatus(dataFile).getPermission());
+        assertTrue("key doesn't exist but should", hbs.exists(validKey));
+
+        // test read
+        BlobStoreFile readpFile = hbs.read(validKey);
+        String readString = IOUtils.toString(readpFile.getInputStream(), "UTF-8");
+        assertEquals("string read from blob doesn't match", testString, readString);
+
+        // test listkeys
+        Iterator<String> keys = hbs.listKeys();
+        assertTrue("blob has one key", keys.hasNext());
+        assertEquals("one key in blobstore", validKey, keys.next());
+
+        // delete
+        hbs.deleteKey(validKey);
+        assertFalse("key not deleted", fs.exists(dataFile));
+        assertFalse("key not deleted", hbs.exists(validKey));
+
+        // Now do multiple
+        String testString2 = "testingblob2";
+        String validKey2= "validkey2";
+
+        // test write
+        pfile = hbs.write(validKey, false);
+        pfile.setMetadata(meta);
+        ios = pfile.getOutputStream();
+        ios.write(testString.getBytes(Charset.forName("UTF-8")));
+        ios.close();
+
+        // test commit creates properly
+        assertTrue("BlobStore key dir wasn't created", fs.exists(fullKeyDir));
+        pfile.commit();
+        assertTrue("blob data not committed", fs.exists(dataFile));
+        assertEquals("BlobStore dir was created with wrong permissions",
+                HdfsBlobStoreFile.BLOBSTORE_FILE_PERMISSION, fs.getFileStatus(dataFile).getPermission());
+        assertTrue("key doesn't exist but should", hbs.exists(validKey));
+
+        // test write again
+        pfile = hbs.write(validKey2, false);
+        pfile.setMetadata(meta);
+        OutputStream ios2 = pfile.getOutputStream();
+        ios2.write(testString2.getBytes(Charset.forName("UTF-8")));
+        ios2.close();
+
+        // test commit second creates properly
+        pfile.commit();
+        Path dataFile2 = new Path(new Path(fullKeyDir, validKey2), BLOBSTORE_DATA);
+        assertTrue("blob data not committed", fs.exists(dataFile2));
+        assertEquals("BlobStore dir was created with wrong permissions",
+                HdfsBlobStoreFile.BLOBSTORE_FILE_PERMISSION, fs.getFileStatus(dataFile2).getPermission());
+        assertTrue("key doesn't exist but should", hbs.exists(validKey2));
+
+        // test listkeys
+        keys = hbs.listKeys();
+        int total = 0;
+        boolean key1Found = false;
+        boolean key2Found = false;
+        while(keys.hasNext()) {
+            total++;
+            String key = keys.next();
+            if (key.equals(validKey)) {
+                key1Found = true;
+            } else if (key.equals(validKey2)) {
+                key2Found = true;
+            } else {
+                fail("Found key that wasn't expected: " + key);
+            }
+        }
+        assertEquals("number of keys is wrong", 2, total);
+        assertTrue("blobstore missing key1", key1Found);
+        assertTrue("blobstore missing key2", key2Found);
+
+        // test read
+        readpFile = hbs.read(validKey);
+        readString = IOUtils.toString(readpFile.getInputStream(), "UTF-8");
+        assertEquals("string read from blob doesn't match", testString, readString);
+
+        // test read
+        readpFile = hbs.read(validKey2);
+        readString = IOUtils.toString(readpFile.getInputStream(), "UTF-8");
+        assertEquals("string read from blob doesn't match", testString2, readString);
+
+        hbs.deleteKey(validKey);
+        assertFalse("key not deleted", hbs.exists(validKey));
+        hbs.deleteKey(validKey2);
+        assertFalse("key not deleted", hbs.exists(validKey2));
+    }
+
+    @Test
+    public void testGetFileLength() throws IOException {
+        FileSystem fs = dfscluster.getFileSystem();
+        Map conf = new HashMap();
+        String validKey = "validkeyBasic";
+        String testString = "testingblob";
+        TestHdfsBlobStoreImpl hbs = new TestHdfsBlobStoreImpl(blobDir, conf, hadoopConf);
+        BlobStoreFile pfile = hbs.write(validKey, false);
+        // Adding metadata to avoid null pointer exception
+        SettableBlobMeta meta = new SettableBlobMeta();
+        meta.set_replication_factor(1);
+        pfile.setMetadata(meta);
+        OutputStream ios = pfile.getOutputStream();
+        ios.write(testString.getBytes(Charset.forName("UTF-8")));
+        ios.close();
+        assertEquals(testString.getBytes(Charset.forName("UTF-8")).length, pfile.getFileLength());
+    }
+}
\ No newline at end of file


[10/17] storm git commit: Blobstore API STORM- 876

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
index d659d57..e066269 100644
--- a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
@@ -14,19 +14,21 @@
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
 (ns backtype.storm.daemon.supervisor
-  (:import [java.io OutputStreamWriter BufferedWriter IOException])
+  (:import [java.io File IOException FileOutputStream])
   (:import [backtype.storm.scheduler ISupervisor]
            [backtype.storm.utils LocalState Time Utils]
            [backtype.storm.daemon Shutdownable]
            [backtype.storm Constants]
            [java.net JarURLConnection]
            [java.net URI]
-           [org.apache.commons.io FileUtils]
-           [java.io File])
+           [org.apache.commons.io FileUtils])
   (:use [backtype.storm config util log timer local-state])
+  (:import [backtype.storm.generated AuthorizationException KeyNotFoundException WorkerResources])
   (:import [backtype.storm.utils VersionInfo])
+  (:import [java.nio.file Files StandardCopyOption])
   (:import [backtype.storm Config])
   (:import [backtype.storm.generated WorkerResources ProfileAction])
+  (:import [backtype.storm.localizer LocalResource])
   (:use [backtype.storm.daemon common])
   (:require [backtype.storm.command [healthcheck :as healthcheck]])
   (:require [backtype.storm.daemon [worker :as worker]]
@@ -44,7 +46,6 @@
 
 (defmulti download-storm-code cluster-mode)
 (defmulti launch-worker (fn [supervisor & _] (cluster-mode (:conf supervisor))))
-(defmulti mk-code-distributor cluster-mode)
 
 (defprotocol SupervisorDaemon
   (get-id [this])
@@ -238,20 +239,21 @@
 (defn- rmr-as-user
   "Launches a process owned by the given user that deletes the given path
   recursively.  Throws RuntimeException if the directory is not removed."
-  [conf id user path]
-  (worker-launcher-and-wait conf
-                            user
-                            ["rmr" path]
-                            :log-prefix (str "rmr " id))
-  (if (exists-file? path)
-    (throw (RuntimeException. (str path " was not deleted")))))
-
-(defn try-cleanup-worker [conf id user]
+  [conf id path]
+  (let [user (Utils/getFileOwner path)]
+    (worker-launcher-and-wait conf
+      user
+      ["rmr" path]
+      :log-prefix (str "rmr " id))
+    (if (exists-file? path)
+      (throw (RuntimeException. (str path " was not deleted"))))))
+
+(defn try-cleanup-worker [conf id]
   (try
     (if (.exists (File. (worker-root conf id)))
       (do
         (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
-          (rmr-as-user conf id user (worker-root conf id))
+          (rmr-as-user conf id (worker-root conf id))
           (do
             (rmr (worker-heartbeats-root conf id))
             ;; this avoids a race condition with worker or subprocess writing pid around same time
@@ -290,11 +292,11 @@
         (worker-launcher-and-wait conf user ["signal" pid "9"] :log-prefix (str "kill -9 " pid))
         (force-kill-process pid))
       (if as-user
-        (rmr-as-user conf id user (worker-pid-path conf id pid))
+        (rmr-as-user conf id (worker-pid-path conf id pid))
         (try
           (rmpath (worker-pid-path conf id pid))
           (catch Exception e)))) ;; on windows, the supervisor may still holds the lock on the worker directory
-    (try-cleanup-worker conf id user))
+    (try-cleanup-worker conf id))
   (log-message "Shut down " (:supervisor-id supervisor) ":" id))
 
 (def SUPERVISOR-ZK-ACLS
@@ -326,16 +328,62 @@
                                          (log-error t "Error when processing event")
                                          (exit-process! 20 "Error when processing an event")
                                          ))
+   :blob-update-timer (mk-timer :kill-fn (defn blob-update-timer
+                                           [t]
+                                           (log-error t "Error when processing event")
+                                           (exit-process! 20 "Error when processing a event"))
+                                :timer-name "blob-update-timer")
+   :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)))))
+
+(defn get-worker-assignment-helper-msg
+  [assignment supervisor port id]
+  (str (pr-str assignment) " for this supervisor " (:supervisor-id supervisor) " on port "
+    port " with id " id))
+
+(defn get-valid-new-worker-ids
+  [conf supervisor reassign-executors 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 "
+                (get-worker-assignment-helper-msg assignment supervisor port 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)
+              [id port])
+            (do
+              (log-message "Missing topology storm code, so can't launch worker with assignment "
+                (get-worker-assignment-helper-msg assignment supervisor port id))
+              nil)))))))
+
 (defn sync-processes [supervisor]
   (let [conf (:conf supervisor)
-        download-lock (:download-lock supervisor)
         ^LocalState local-state (:local-state supervisor)
         storm-cluster-state (:storm-cluster-state supervisor)
         assigned-executors (defaulted (ls-local-assignments local-state) {})
@@ -349,8 +397,7 @@
         new-worker-ids (into
                         {}
                         (for [port (keys reassign-executors)]
-                          [port (uuid)]))
-        ]
+                          [port (uuid)]))]
     ;; 1. to kill are those in allocated that are dead or disallowed
     ;; 2. kill the ones that should be dead
     ;;     - read pids, kill -9 and individually remove file
@@ -371,67 +418,14 @@
          ". Current supervisor time: " now
          ". 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)))
-    ))
+        (shutdown-worker supervisor id)))
+    (let [valid-new-worker-ids (get-valid-new-worker-ids conf supervisor reassign-executors new-worker-ids)]
+      (ls-approved-workers! local-state
+                        (merge
+                          (select-keys (ls-approved-workers local-state)
+                            (keys keepers))
+                          valid-new-worker-ids))
+      (wait-for-workers-launch conf (keys valid-new-worker-ids)))))
 
 (defn assigned-storm-ids-from-port-assignments [assignment]
   (->> assignment
@@ -454,10 +448,80 @@
       (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
+  "Check for the files exists to avoid supervisor crashing
+   Also makes sure there is no necessity for locking"
+  [conf localizer assigned-storm-ids all-downloaded-storm-ids]
+  (remove nil?
+    (into #{}
+      (for [storm-id all-downloaded-storm-ids
+            :when (contains? assigned-storm-ids storm-id)]
+        (when-not (required-topo-files-exist? conf storm-id)
+          (log-debug "Files not present in topology directory")
+          (rm-topo-files conf storm-id localizer false)
+          storm-id)))))
+
 (defn mk-synchronize-supervisor [supervisor sync-processes event-manager processes-event-manager]
   (fn this []
     (let [conf (:conf supervisor)
-          download-lock (:download-lock supervisor)
           storm-cluster-state (:storm-cluster-state supervisor)
           ^ISupervisor isupervisor (:isupervisor supervisor)
           ^LocalState local-state (:local-state supervisor)
@@ -468,7 +532,7 @@
            versions :versions}
           (assignments-snapshot storm-cluster-state sync-callback assignment-versions)
           storm-code-map (read-storm-code-locations assignments-snapshot)
-          downloaded-storm-ids (set (read-downloaded-storm-ids conf))
+          all-downloaded-storm-ids (set (read-downloaded-storm-ids conf))
           existing-assignment (ls-local-assignments local-state)
           all-assignment (read-assignments assignments-snapshot
                                            (:assignment-id supervisor)
@@ -476,14 +540,20 @@
                                            (:sync-retry supervisor))
           new-assignment (->> all-assignment
                               (filter-key #(.confirmAssigned isupervisor %)))
-          assigned-storm-ids (assigned-storm-ids-from-port-assignments new-assignment)]
+          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 "Storm Ids Profiler Actions" storm-id->profiler-actions)
-
+      (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)
+      (log-debug "Storm Ids Profiler Actions " storm-id->profiler-actions)
       ;; download code first
       ;; This might take awhile
       ;;   - should this be done separately from usual monitoring?
@@ -491,7 +561,9 @@
       (doseq [[storm-id master-code-dir] storm-code-map]
         (when (and (not (downloaded-storm-ids storm-id))
                    (assigned-storm-ids storm-id))
-          (download-storm-code conf storm-id master-code-dir supervisor download-lock)))
+          (log-message "Downloading code for storm id " storm-id)
+          (download-storm-code conf storm-id master-code-dir localizer)
+          (log-message "Finished downloading code for storm id " storm-id)))
 
       (log-debug "Writing new assignment "
                  (pr-str new-assignment))
@@ -510,22 +582,52 @@
       ;; synchronize-supervisor doesn't try to launch workers for which the
       ;; resources don't exist
       (if on-windows? (shutdown-disallowed-workers supervisor))
-      (doseq [storm-id downloaded-storm-ids]
+      (doseq [storm-id all-downloaded-storm-ids]
         (when-not (storm-code-map storm-id)
           (log-message "Removing code for storm id "
                        storm-id)
-          (try
-            (rmr (supervisor-stormdist-root conf storm-id))
-            (catch Exception e (log-message (.getMessage e))))
-          ))
-      (.add processes-event-manager sync-processes)
-      )))
+          (rm-topo-files conf storm-id localizer true)))
+      (.add processes-event-manager sync-processes))))
 
 (defn mk-supervisor-capacities
   [conf]
   {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)
+        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 []
+    (try
+      (let [conf (:conf supervisor)
+            downloaded-storm-ids (set (read-downloaded-storm-ids conf))
+            new-assignment @(:curr-assignment supervisor)
+            assigned-storm-ids (assigned-storm-ids-from-port-assignments new-assignment)]
+        (doseq [topology-id downloaded-storm-ids]
+          (let [storm-root (supervisor-stormdist-root conf topology-id)]
+            (when (assigned-storm-ids topology-id)
+              (log-debug "Checking Blob updates for storm topology id " topology-id " With target_dir: " storm-root)
+              (update-blobs-for-topology! conf topology-id (:localizer supervisor))))))
+      (catch Exception e
+        (log-error e "Error updating blobs, will retry again later")))))
+
 (defn jvm-cmd [cmd]
   (let [java-home (.get (System/getenv) "JAVA_HOME")]
     (if (nil? java-home)
@@ -650,6 +752,8 @@
         [event-manager processes-event-manager :as managers] [(event/event-manager false) (event/event-manager false)]
         sync-processes (partial sync-processes supervisor)
         synchronize-supervisor (mk-synchronize-supervisor supervisor sync-processes event-manager processes-event-manager)
+        synchronize-blobs-fn (update-blobs-for-all-topologies-fn supervisor)
+        downloaded-storm-ids (set (read-downloaded-storm-ids conf))
         run-profiler-actions-fn (mk-run-profiler-actions-for-all-topologies supervisor)
         heartbeat-fn (fn [] (.supervisor-heartbeat!
                                (:storm-cluster-state supervisor)
@@ -671,6 +775,12 @@
                         0
                         (conf SUPERVISOR-HEARTBEAT-FREQUENCY-SECS)
                         heartbeat-fn)
+    (doseq [storm-id downloaded-storm-ids]
+      (add-blob-references (:localizer supervisor) storm-id
+        conf))
+    ;; do this after adding the references so we don't try to clean things being used
+    (.startCleaner (:localizer supervisor))
+
     (when (conf SUPERVISOR-ENABLE)
       ;; This isn't strictly necessary, but it doesn't hurt and ensures that the machine stays up
       ;; to date even if callbacks don't all work exactly right
@@ -679,6 +789,13 @@
                           0
                           (conf SUPERVISOR-MONITOR-FREQUENCY-SECS)
                           (fn [] (.add processes-event-manager sync-processes)))
+
+      ;; Blob update thread. Starts with 30 seconds delay, every 30 seconds
+      (schedule-recurring (:blob-update-timer supervisor)
+                          30
+                          30
+                          (fn [] (.add event-manager synchronize-blobs-fn)))
+
       (schedule-recurring (:event-timer supervisor)
                           (* 60 5)
                           (* 60 5)
@@ -689,6 +806,7 @@
                                        (doseq [id ids]
                                          (shutdown-worker supervisor id))
                                        (throw (RuntimeException. "Supervisor failed health check. Exiting.")))))))
+
       ;; Launch a thread that Runs profiler commands . Starts with 30 seconds delay, every 30 seconds
       (schedule-recurring (:event-timer supervisor)
                           30
@@ -702,8 +820,10 @@
                (reset! (:active supervisor) false)
                (cancel-timer (:heartbeat-timer supervisor))
                (cancel-timer (:event-timer supervisor))
+               (cancel-timer (:blob-update-timer supervisor))
                (.shutdown event-manager)
                (.shutdown processes-event-manager)
+               (.shutdown (:localizer supervisor))
                (.disconnect (:storm-cluster-state supervisor)))
      SupervisorDaemon
      (get-conf [this]
@@ -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)))
+      (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-not (empty? file-names)
+      (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/getClientBlobStoreForSupervisor conf)]
+    (FileUtils/forceMkdir (File. tmproot))
+    (if-not on-windows?
+      (Utils/restrictPermissions tmproot)
+      (if (conf SUPERVISOR-RUN-WORKER-AS-USER)
+        (throw-runtime (str "ERROR: Windows doesn't implement setting the correct permissions"))))
+    (Utils/downloadResourcesAsSupervisor (master-stormjar-key storm-id)
+      (supervisor-stormjar-path tmproot) blobstore)
+    (Utils/downloadResourcesAsSupervisor (master-stormcode-key storm-id)
+      (supervisor-stormcode-path tmproot) blobstore)
+    (Utils/downloadResourcesAsSupervisor (master-stormconf-key storm-id)
+      (supervisor-stormconf-path tmproot) blobstore)
+    (.shutdown blobstore)
+    (extract-dir-from-jar (supervisor-stormjar-path tmproot) RESOURCES-SUBDIR tmproot)
+    (download-blobs-for-topology! conf (supervisor-stormconf-path tmproot) localizer
+      tmproot)
+    (if (download-blobs-for-topology-succeed? (supervisor-stormconf-path tmproot) tmproot)
+      (do
+        (log-message "Successfully downloaded blob resources for storm-id " storm-id)
+        (FileUtils/forceMkdir (File. stormroot))
+        (Files/move (.toPath (File. tmproot)) (.toPath (File. stormroot))
+          (doto (make-array StandardCopyOption 1) (aset 0 StandardCopyOption/ATOMIC_MOVE)))
+        (setup-storm-code-dir conf (read-supervisor-storm-conf conf storm-id) stormroot))
+      (do
+        (log-message "Failed to download blob resources for storm-id " storm-id)
+        (rmr tmproot)))))
 
 (defn write-log-metadata-to-yaml-file! [storm-id port data conf]
   (let [file (get-log-metadata-file conf storm-id port)]
@@ -781,11 +964,6 @@
                                              (storm-conf TOPOLOGY-USERS)))))}]
     (write-log-metadata-to-yaml-file! storm-id port data conf)))
 
-(defmethod mk-code-distributor :distributed [conf]
-  (let [code-distributor (new-instance (conf STORM-CODE-DISTRIBUTOR-CLASS))]
-    (.prepare code-distributor conf)
-    code-distributor))
-
 (defn jlp [stormroot conf]
   (let [resource-root (str stormroot File/separator RESOURCES-SUBDIR)
         os (clojure.string/replace (System/getProperty "os.name") #"\s+" "_")
@@ -811,6 +989,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 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))))
+
 (defn create-artifacts-link
   "Create a symlink from workder directory to its port artifacts directory"
   [conf storm-id port worker-id]
@@ -913,6 +1106,7 @@
                        (add-dead-worker worker-id))
             worker-dir (worker-root conf worker-id)]
         (remove-dead-worker worker-id)
+        (create-blobstore-links conf storm-id worker-id)
         (if run-worker-as-user
           (worker-launcher conf user ["worker" worker-dir (write-script worker-dir command :environment topology-worker-environment)] :log-prefix log-prefix :exit-code-callback callback :directory (File. worker-dir))
           (launch-process command :environment topology-worker-environment :log-prefix log-prefix :exit-code-callback callback :directory (File. worker-dir)))
@@ -927,31 +1121,31 @@
        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
+        (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 " (str url) " to " target-dir)
+          (FileUtils/copyDirectory (File. (.getFile url)) (File. target-dir)))))))
 
 (defmethod launch-worker
     :local [supervisor storm-id port worker-id mem-onheap]

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/clj/backtype/storm/testing.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/testing.clj b/storm-core/src/clj/backtype/storm/testing.clj
index 2c98b07..c552519 100644
--- a/storm-core/src/clj/backtype/storm/testing.clj
+++ b/storm-core/src/clj/backtype/storm/testing.clj
@@ -126,7 +126,8 @@
                             ZMQ-LINGER-MILLIS 0
                             TOPOLOGY-ENABLE-MESSAGE-TIMEOUTS false
                             TOPOLOGY-TRIDENT-BATCH-EMIT-INTERVAL-MILLIS 50
-                            STORM-CLUSTER-MODE "local"}
+                            STORM-CLUSTER-MODE "local"
+                            BLOBSTORE-SUPERUSER (System/getProperty "user.name")}
                            (if-not (contains? daemon-conf STORM-ZOOKEEPER-SERVERS)
                              {STORM-ZOOKEEPER-PORT zk-port
                               STORM-ZOOKEEPER-SERVERS ["localhost"]})
@@ -628,7 +629,7 @@
           track-id (-> tracked-topology :cluster ::track-id)
           waiting? (fn []
                      (or (not= target (global-amt track-id "spout-emitted"))
-                         (not= (global-amt track-id "transferred")                                 
+                         (not= (global-amt track-id "transferred")
                                (global-amt track-id "processed"))))]
       (while-timeout timeout-ms (waiting?)
                      ;; (println "Spout emitted: " (global-amt track-id "spout-emitted"))

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/clj/backtype/storm/util.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/util.clj b/storm-core/src/clj/backtype/storm/util.clj
index 9b22e70..cbe5bf9 100644
--- a/storm-core/src/clj/backtype/storm/util.clj
+++ b/storm-core/src/clj/backtype/storm/util.clj
@@ -22,6 +22,8 @@
   (:import [backtype.storm Config])
   (:import [backtype.storm.utils Time Container ClojureTimerTask Utils
             MutableObject MutableInt])
+  (:import [backtype.storm.security.auth NimbusPrincipal])
+  (:import [javax.security.auth Subject])
   (:import [java.util UUID Random ArrayList List Collections])
   (:import [java.util.zip ZipFile])
   (:import [java.util.concurrent.locks ReentrantReadWriteLock])
@@ -1099,7 +1101,19 @@
     (assoc coll k (apply str (repeat (count (coll k)) "#")))
     coll))
 
-(defn log-thrift-access [request-id remoteAddress principal operation]
+(defn log-thrift-access
+  [request-id remoteAddress principal operation]
   (doto
     (ThriftAccessLogger.)
     (.log (str "Request ID: " request-id " access from: " remoteAddress " principal: " principal " operation: " operation))))
+
+(def DISALLOWED-KEY-NAME-STRS #{"/" "." ":" "\\"})
+
+(defn validate-key-name!
+  [name]
+  (if (some #(.contains name %) DISALLOWED-KEY-NAME-STRS)
+    (throw (RuntimeException.
+             (str "Key name cannot contain any of the following: " (pr-str DISALLOWED-KEY-NAME-STRS))))
+    (if (clojure.string/blank? name)
+      (throw (RuntimeException.
+               ("Key name cannot be blank"))))))

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/clj/backtype/storm/zookeeper.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/zookeeper.clj b/storm-core/src/clj/backtype/storm/zookeeper.clj
index 26def33..c91ffa4 100644
--- a/storm-core/src/clj/backtype/storm/zookeeper.clj
+++ b/storm-core/src/clj/backtype/storm/zookeeper.clj
@@ -114,6 +114,7 @@
       (try-cause  (.. zk (delete) (deletingChildrenIfNeeded) (forPath (normalize-path path)))
                   (catch KeeperException$NoNodeException e
                     ;; do nothing
+                    (log-message "exception" e)
                   )
                   (catch Exception e (throw (wrap-in-runtime e)))))))
 
@@ -129,7 +130,6 @@
           ))
       )))
 
-
 (defn sync-path
   [^CuratorFramework zk ^String path]
   (try
@@ -186,6 +186,19 @@
       (.. zk (getChildren) (forPath (normalize-path path))))
     (catch Exception e (throw (wrap-in-runtime e)))))
 
+(defn delete-node-blobstore
+  "Deletes the state inside the zookeeper for a key, for which the
+   contents of the key starts with nimbus host port information"
+  [^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]
+      (when (.startsWith child host-port-info)
+        (log-debug "delete-node " "child" child)
+        (delete-node zk (str parent-path "/" child))))))
+
 (defn set-data
   [^CuratorFramework zk ^String path ^bytes data]
   (try
@@ -232,22 +245,10 @@
 (defn leader-latch-listener-impl
   "Leader latch listener that will be invoked when we either gain or lose leadership"
   [conf zk leader-latch]
-  (let [hostname (.getCanonicalHostName (InetAddress/getLocalHost))
-        STORMS-ROOT (str (conf STORM-ZOOKEEPER-ROOT) "/storms")]
+  (let [hostname (.getCanonicalHostName (InetAddress/getLocalHost))]
     (reify LeaderLatchListener
       (^void isLeader[this]
-        (log-message (str hostname " gained leadership, checking if it has all the topology code locally."))
-        (let [active-topology-ids (set (get-children zk STORMS-ROOT false))
-              local-topology-ids (set (.list (File. (master-stormdist-root conf))))
-              diff-topology (first (set-delta active-topology-ids local-topology-ids))]
-        (log-message "active-topology-ids [" (clojure.string/join "," active-topology-ids)
-                          "] local-topology-ids [" (clojure.string/join "," local-topology-ids)
-                          "] diff-topology [" (clojure.string/join "," diff-topology) "]")
-        (if (empty? diff-topology)
-          (log-message "Accepting leadership, all active topology found localy.")
-          (do
-            (log-message "code for all active topologies not available locally, giving up leadership.")
-            (.close leader-latch)))))
+        (log-message (str hostname " gained leadership")))
       (^void notLeader[this]
         (log-message (str hostname " lost leadership."))))))
 

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java
index 89422f6..b663dcb 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -1056,6 +1056,122 @@ public class Config extends HashMap<String, Object> {
     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";
+
+    /**
+     * 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";
+
+    /**
+     * Maximum number of retries a supervisor is allowed to make for downloading a blob.
+     */
+    @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";
+
+    /**
+     * What blobstore implementation nimbus should use.
+     */
+    @isString
+    public static final String NIMBUS_BLOBSTORE = "nimbus.blobstore.class";
+
+    /**
+     * During operations with the blob store, via master, how long a connection
+     * is idle before nimbus considers it dead and drops the session and any
+     * associated connections.
+     */
+    @isPositiveNumber
+    @isInteger
+    public static final String NIMBUS_BLOBSTORE_EXPIRATION_SECS = "nimbus.blobstore.expiration.secs";
+
+    /**
+     * A map with blobstore keys mapped to each filename the worker will have access to in the
+     * launch directory to the blob by local file name and uncompress flag. Both localname and
+     * uncompress flag are optional. It uses the key is localname is not specified. Each topology
+     * will have different map of blobs.  Example: topology.blobstore.map: {"blobstorekey" :
+     * {"localname": "myblob", "uncompress": false}, "blobstorearchivekey" :
+     * {"localname": "myarchive", "uncompress": true}}
+     */
+    @CustomValidator(validatorClass = MapOfStringToMapOfStringToObjectValidator.class)
+    public static final String TOPOLOGY_BLOBSTORE_MAP = "topology.blobstore.map";
+
+    /**
      * A number representing the maximum number of workers any single topology can acquire.
      */
     @isInteger
@@ -1847,13 +1963,6 @@ public class Config extends HashMap<String, Object> {
     public static final String TOPOLOGY_DISRUPTOR_BATCH_TIMEOUT_MILLIS="topology.disruptor.batch.timeout.millis";
 
     /**
-     * Which implementation of {@link backtype.storm.codedistributor.ICodeDistributor} should be used by storm for code
-     * distribution.
-     */
-    @isString
-    public static final String STORM_CODE_DISTRIBUTOR_CLASS = "storm.codedistributor.class";
-
-    /**
      * Minimum number of nimbus hosts where the code must be replicated before leader nimbus
      * is allowed to perform topology activation tasks like setting up heartbeats/assignments
      * and marking the topology as active. default is 0.

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/blobstore/AtomicOutputStream.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/blobstore/AtomicOutputStream.java b/storm-core/src/jvm/backtype/storm/blobstore/AtomicOutputStream.java
new file mode 100644
index 0000000..f35b7a7
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/blobstore/AtomicOutputStream.java
@@ -0,0 +1,32 @@
+/**
+ * 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.IOException;
+import java.io.OutputStream;
+
+/**
+ * An output stream where all of the data is committed on close,
+ * or can be canceled with cancel.
+ */
+public abstract class AtomicOutputStream extends OutputStream {
+    /**
+     * Cancel all of the writes associated with this stream and close it.
+     */ 
+    public abstract void cancel() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/blobstore/BlobKeySequenceInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/blobstore/BlobKeySequenceInfo.java b/storm-core/src/jvm/backtype/storm/blobstore/BlobKeySequenceInfo.java
new file mode 100644
index 0000000..53cfa15
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/blobstore/BlobKeySequenceInfo.java
@@ -0,0 +1,40 @@
+/**
+ * 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 class BlobKeySequenceInfo {
+    private String nimbusHostPort;
+    private String sequenceNumber;
+
+    public void setNimbusHostPort(String nimbusHostPort) {
+     this.nimbusHostPort = nimbusHostPort;
+    }
+
+    public void setSequenceNumber(String sequenceNumber) {
+        this.sequenceNumber = sequenceNumber;
+    }
+
+    public String getNimbusHostPort() {
+        return nimbusHostPort;
+    }
+
+    public String getSequenceNumber() {
+        return sequenceNumber;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/blobstore/BlobStore.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/blobstore/BlobStore.java b/storm-core/src/jvm/backtype/storm/blobstore/BlobStore.java
new file mode 100644
index 0000000..a714b76
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/blobstore/BlobStore.java
@@ -0,0 +1,445 @@
+/**
+ * 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.apache.commons.lang.StringUtils;
+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 {
+    private 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 (StringUtils.isEmpty(key) || "..".equals(key) || ".".equals(key) || !KEY_PATTERN.matcher(key).matches()) {
+            LOG.error("'{}' does not appear to be valid {}", key, KEY_PATTERN);
+            throw new IllegalArgumentException(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) {
+            if (out !=null) {
+                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 having read
+     * privilege for 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 having
+     * the read privilege for 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 implementation used for reading the
+     * metadata and data information.
+     */
+    protected class BlobStoreFileOutputStream extends AtomicOutputStream {
+        private BlobStoreFile part;
+        private OutputStream out;
+
+        public BlobStoreFileOutputStream(BlobStoreFile part) throws IOException {
+            this.part = part;
+            this.out = part.getOutputStream();
+        }
+
+        @Override
+        public void close() throws IOException {
+            try {
+                //close means commit
+                out.close();
+                part.commit();
+            } catch (IOException | RuntimeException e) {
+                cancel();
+                throw e;
+            }
+        }
+
+        @Override
+        public void cancel() throws IOException {
+            try {
+                out.close();
+            } finally {
+                part.cancel();
+            }
+        }
+
+        @Override
+        public void write(int b) throws IOException {
+            out.write(b);
+        }
+
+        @Override
+        public void write(byte []b) throws IOException {
+            out.write(b);
+        }
+
+        @Override
+        public void write(byte []b, int offset, int len) throws IOException {
+            out.write(b, offset, len);
+        }
+    }
+
+    /**
+     * Input stream implementation used for writing
+     * both the metadata containing the acl information
+     * and the blob data.
+     */
+    protected class BlobStoreFileInputStream extends InputStreamWithMeta {
+        private BlobStoreFile part;
+        private InputStream in;
+
+        public BlobStoreFileInputStream(BlobStoreFile part) throws IOException {
+            this.part = part;
+            this.in = part.getInputStream();
+        }
+
+        @Override
+        public long getVersion() throws IOException {
+            return part.getModTime();
+        }
+
+        @Override
+        public int read() throws IOException {
+            return in.read();
+        }
+
+        @Override
+        public int read(byte[] b, int off, int len) throws IOException {
+            return in.read(b, off, len);
+        }
+
+        @Override
+        public int read(byte[] b) throws IOException {
+            return in.read(b);
+        }
+
+        @Override
+        public int available() throws IOException {
+            return in.available();
+        }
+
+        @Override
+        public long getFileLength() throws IOException {
+            return part.getFileLength();
+        }
+    }
+
+    /**
+     * Blob store implements its own version of iterator
+     * to list the blobs
+     */
+    public static class KeyTranslationIterator implements Iterator<String> {
+        private Iterator<String> it = null;
+        private String next = null;
+        private String prefix = null;
+
+        public KeyTranslationIterator(Iterator<String> it, String prefix) throws IOException {
+            this.it = it;
+            this.prefix = prefix;
+            primeNext();
+        }
+
+        private void primeNext() {
+            next = null;
+            while (it.hasNext()) {
+                String tmp = it.next();
+                if (tmp.startsWith(prefix)) {
+                    next = tmp.substring(prefix.length());
+                    return;
+                }
+            }
+        }
+
+        @Override
+        public boolean hasNext() {
+            return next != null;
+        }
+
+        @Override
+        public String next() {
+            if (!hasNext()) {
+                throw new NoSuchElementException();
+            }
+            String current = next;
+            primeNext();
+            return current;
+        }
+
+        @Override
+        public void remove() {
+            throw new UnsupportedOperationException("Delete Not Supported");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/blobstore/BlobStoreAclHandler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/blobstore/BlobStoreAclHandler.java b/storm-core/src/jvm/backtype/storm/blobstore/BlobStoreAclHandler.java
new file mode 100644
index 0000000..c0c4e5c
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/blobstore/BlobStoreAclHandler.java
@@ -0,0 +1,399 @@
+/**
+ * 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.apache.commons.lang.StringUtils;
+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) {
+        StringBuilder ret = new StringBuilder();
+        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) {
+        StringBuilder ret = new StringBuilder();
+        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 (!StringUtils.isEmpty(aclUser) && !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;
+        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) {
+        return isNimbus(who) || isAdmin(who) || isSupervisor(who,mask);
+    }
+
+    /**
+     * 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 cumulative 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 at least the set of permissions
+     * mentioned in the mask.
+     * @param acl ACL for the key.
+     * @param mask mask holds the cumulative 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) {
+        StringBuilder b = new StringBuilder();
+        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() == (READ|WRITE|ADMIN)) {
+                isWorldEverything = true;
+                break;
+            }
+        }
+        return isWorldEverything;
+    }
+
+    private void fixACLsForUser(List<AccessControl> acls, String user, int mask) {
+        boolean foundUserACL = false;
+        for (AccessControl control : acls) {
+            if (control.get_type() == AccessControlType.USER && control.get_name().equals(user)) {
+                int currentAccess = control.get_access();
+                if ((currentAccess & mask) != mask) {
+                    control.set_access(currentAccess | mask);
+                }
+                foundUserACL = true;
+                break;
+            }
+        }
+        if (!foundUserACL) {
+            AccessControl userACL = new AccessControl();
+            userACL.set_type(AccessControlType.USER);
+            userACL.set_name(user);
+            userACL.set_access(mask);
+            acls.add(userACL);
+        }
+    }
+
+    private Set<String> getUserNamesFromSubject(Subject who) {
+        Set<String> user = new HashSet<String>();
+        if (who != null) {
+            for(Principal p: who.getPrincipals()) {
+                user.add(_ptol.toLocal(p));
+            }
+        }
+        return user;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/blobstore/BlobStoreFile.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/blobstore/BlobStoreFile.java b/storm-core/src/jvm/backtype/storm/blobstore/BlobStoreFile.java
new file mode 100644
index 0000000..22ccf97
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/blobstore/BlobStoreFile.java
@@ -0,0 +1,50 @@
+/**
+ * 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.generated.SettableBlobMeta;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.regex.Pattern;
+
+/**
+ * Provides an base implementation for creating a blobstore based on file backed storage.
+ */
+public abstract class BlobStoreFile {
+    public static final Logger LOG = LoggerFactory.getLogger(BlobStoreFile.class);
+
+    protected static final String TMP_EXT = ".tmp";
+    protected static final Pattern TMP_NAME_PATTERN = Pattern.compile("^\\d+\\" + TMP_EXT + "$");
+    protected static final String BLOBSTORE_DATA_FILE = "data";
+
+    public abstract void delete() throws IOException;
+    public abstract String getKey();
+    public abstract boolean isTmp();
+    public abstract void setMetadata(SettableBlobMeta meta);
+    public abstract SettableBlobMeta getMetadata();
+    public abstract long getModTime() throws IOException;
+    public abstract InputStream getInputStream() throws IOException;
+    public abstract OutputStream getOutputStream() throws IOException;
+    public abstract void commit() throws IOException;
+    public abstract void cancel() throws IOException;
+    public abstract long getFileLength() throws IOException;
+}


[08/17] storm git commit: Blobstore API STORM- 876

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/generated/AccessControl.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/AccessControl.java b/storm-core/src/jvm/backtype/storm/generated/AccessControl.java
new file mode 100644
index 0000000..2209168
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/AccessControl.java
@@ -0,0 +1,627 @@
+/**
+ * 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.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class AccessControl implements org.apache.thrift.TBase<AccessControl, AccessControl._Fields>, java.io.Serializable, Cloneable, Comparable<AccessControl> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AccessControl");
+
+  private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)1);
+  private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField ACCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("access", org.apache.thrift.protocol.TType.I32, (short)3);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new AccessControlStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new AccessControlTupleSchemeFactory());
+  }
+
+  private AccessControlType type; // required
+  private String name; // optional
+  private int access; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    /**
+     * 
+     * @see AccessControlType
+     */
+    TYPE((short)1, "type"),
+    NAME((short)2, "name"),
+    ACCESS((short)3, "access");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // TYPE
+          return TYPE;
+        case 2: // NAME
+          return NAME;
+        case 3: // ACCESS
+          return ACCESS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __ACCESS_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.NAME};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, AccessControlType.class)));
+    tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.ACCESS, new org.apache.thrift.meta_data.FieldMetaData("access", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(AccessControl.class, metaDataMap);
+  }
+
+  public AccessControl() {
+  }
+
+  public AccessControl(
+    AccessControlType type,
+    int access)
+  {
+    this();
+    this.type = type;
+    this.access = access;
+    set_access_isSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public AccessControl(AccessControl other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_type()) {
+      this.type = other.type;
+    }
+    if (other.is_set_name()) {
+      this.name = other.name;
+    }
+    this.access = other.access;
+  }
+
+  public AccessControl deepCopy() {
+    return new AccessControl(this);
+  }
+
+  @Override
+  public void clear() {
+    this.type = null;
+    this.name = null;
+    set_access_isSet(false);
+    this.access = 0;
+  }
+
+  /**
+   * 
+   * @see AccessControlType
+   */
+  public AccessControlType get_type() {
+    return this.type;
+  }
+
+  /**
+   * 
+   * @see AccessControlType
+   */
+  public void set_type(AccessControlType type) {
+    this.type = type;
+  }
+
+  public void unset_type() {
+    this.type = null;
+  }
+
+  /** Returns true if field type is set (has been assigned a value) and false otherwise */
+  public boolean is_set_type() {
+    return this.type != null;
+  }
+
+  public void set_type_isSet(boolean value) {
+    if (!value) {
+      this.type = null;
+    }
+  }
+
+  public String get_name() {
+    return this.name;
+  }
+
+  public void set_name(String name) {
+    this.name = name;
+  }
+
+  public void unset_name() {
+    this.name = null;
+  }
+
+  /** Returns true if field name is set (has been assigned a value) and false otherwise */
+  public boolean is_set_name() {
+    return this.name != null;
+  }
+
+  public void set_name_isSet(boolean value) {
+    if (!value) {
+      this.name = null;
+    }
+  }
+
+  public int get_access() {
+    return this.access;
+  }
+
+  public void set_access(int access) {
+    this.access = access;
+    set_access_isSet(true);
+  }
+
+  public void unset_access() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ACCESS_ISSET_ID);
+  }
+
+  /** Returns true if field access is set (has been assigned a value) and false otherwise */
+  public boolean is_set_access() {
+    return EncodingUtils.testBit(__isset_bitfield, __ACCESS_ISSET_ID);
+  }
+
+  public void set_access_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ACCESS_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TYPE:
+      if (value == null) {
+        unset_type();
+      } else {
+        set_type((AccessControlType)value);
+      }
+      break;
+
+    case NAME:
+      if (value == null) {
+        unset_name();
+      } else {
+        set_name((String)value);
+      }
+      break;
+
+    case ACCESS:
+      if (value == null) {
+        unset_access();
+      } else {
+        set_access((Integer)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TYPE:
+      return get_type();
+
+    case NAME:
+      return get_name();
+
+    case ACCESS:
+      return get_access();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case TYPE:
+      return is_set_type();
+    case NAME:
+      return is_set_name();
+    case ACCESS:
+      return is_set_access();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof AccessControl)
+      return this.equals((AccessControl)that);
+    return false;
+  }
+
+  public boolean equals(AccessControl that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_type = true && this.is_set_type();
+    boolean that_present_type = true && that.is_set_type();
+    if (this_present_type || that_present_type) {
+      if (!(this_present_type && that_present_type))
+        return false;
+      if (!this.type.equals(that.type))
+        return false;
+    }
+
+    boolean this_present_name = true && this.is_set_name();
+    boolean that_present_name = true && that.is_set_name();
+    if (this_present_name || that_present_name) {
+      if (!(this_present_name && that_present_name))
+        return false;
+      if (!this.name.equals(that.name))
+        return false;
+    }
+
+    boolean this_present_access = true;
+    boolean that_present_access = true;
+    if (this_present_access || that_present_access) {
+      if (!(this_present_access && that_present_access))
+        return false;
+      if (this.access != that.access)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_type = true && (is_set_type());
+    list.add(present_type);
+    if (present_type)
+      list.add(type.getValue());
+
+    boolean present_name = true && (is_set_name());
+    list.add(present_name);
+    if (present_name)
+      list.add(name);
+
+    boolean present_access = true;
+    list.add(present_access);
+    if (present_access)
+      list.add(access);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(AccessControl other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_type()).compareTo(other.is_set_type());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_type()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, other.type);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_name()).compareTo(other.is_set_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_access()).compareTo(other.is_set_access());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_access()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.access, other.access);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("AccessControl(");
+    boolean first = true;
+
+    sb.append("type:");
+    if (this.type == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.type);
+    }
+    first = false;
+    if (is_set_name()) {
+      if (!first) sb.append(", ");
+      sb.append("name:");
+      if (this.name == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.name);
+      }
+      first = false;
+    }
+    if (!first) sb.append(", ");
+    sb.append("access:");
+    sb.append(this.access);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_type()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'type' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_access()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'access' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class AccessControlStandardSchemeFactory implements SchemeFactory {
+    public AccessControlStandardScheme getScheme() {
+      return new AccessControlStandardScheme();
+    }
+  }
+
+  private static class AccessControlStandardScheme extends StandardScheme<AccessControl> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, AccessControl struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // TYPE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.type = backtype.storm.generated.AccessControlType.findByValue(iprot.readI32());
+              struct.set_type_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.name = iprot.readString();
+              struct.set_name_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // ACCESS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.access = iprot.readI32();
+              struct.set_access_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, AccessControl struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.type != null) {
+        oprot.writeFieldBegin(TYPE_FIELD_DESC);
+        oprot.writeI32(struct.type.getValue());
+        oprot.writeFieldEnd();
+      }
+      if (struct.name != null) {
+        if (struct.is_set_name()) {
+          oprot.writeFieldBegin(NAME_FIELD_DESC);
+          oprot.writeString(struct.name);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldBegin(ACCESS_FIELD_DESC);
+      oprot.writeI32(struct.access);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class AccessControlTupleSchemeFactory implements SchemeFactory {
+    public AccessControlTupleScheme getScheme() {
+      return new AccessControlTupleScheme();
+    }
+  }
+
+  private static class AccessControlTupleScheme extends TupleScheme<AccessControl> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, AccessControl struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeI32(struct.type.getValue());
+      oprot.writeI32(struct.access);
+      BitSet optionals = new BitSet();
+      if (struct.is_set_name()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_name()) {
+        oprot.writeString(struct.name);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, AccessControl struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.type = backtype.storm.generated.AccessControlType.findByValue(iprot.readI32());
+      struct.set_type_isSet(true);
+      struct.access = iprot.readI32();
+      struct.set_access_isSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.name = iprot.readString();
+        struct.set_name_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/generated/AccessControlType.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/AccessControlType.java b/storm-core/src/jvm/backtype/storm/generated/AccessControlType.java
new file mode 100644
index 0000000..3a9aa70
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/AccessControlType.java
@@ -0,0 +1,62 @@
+/**
+ * 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.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+public enum AccessControlType implements org.apache.thrift.TEnum {
+  OTHER(1),
+  USER(2);
+
+  private final int value;
+
+  private AccessControlType(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static AccessControlType findByValue(int value) { 
+    switch (value) {
+      case 1:
+        return OTHER;
+      case 2:
+        return USER;
+      default:
+        return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/generated/Assignment.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/Assignment.java b/storm-core/src/jvm/backtype/storm/generated/Assignment.java
index cc9bb19..cf59c05 100644
--- a/storm-core/src/jvm/backtype/storm/generated/Assignment.java
+++ b/storm-core/src/jvm/backtype/storm/generated/Assignment.java
@@ -787,15 +787,15 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           case 2: // NODE_HOST
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map532 = iprot.readMapBegin();
-                struct.node_host = new HashMap<String,String>(2*_map532.size);
-                String _key533;
-                String _val534;
-                for (int _i535 = 0; _i535 < _map532.size; ++_i535)
+                org.apache.thrift.protocol.TMap _map548 = iprot.readMapBegin();
+                struct.node_host = new HashMap<String,String>(2*_map548.size);
+                String _key549;
+                String _val550;
+                for (int _i551 = 0; _i551 < _map548.size; ++_i551)
                 {
-                  _key533 = iprot.readString();
-                  _val534 = iprot.readString();
-                  struct.node_host.put(_key533, _val534);
+                  _key549 = iprot.readString();
+                  _val550 = iprot.readString();
+                  struct.node_host.put(_key549, _val550);
                 }
                 iprot.readMapEnd();
               }
@@ -807,26 +807,26 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           case 3: // EXECUTOR_NODE_PORT
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map536 = iprot.readMapBegin();
-                struct.executor_node_port = new HashMap<List<Long>,NodeInfo>(2*_map536.size);
-                List<Long> _key537;
-                NodeInfo _val538;
-                for (int _i539 = 0; _i539 < _map536.size; ++_i539)
+                org.apache.thrift.protocol.TMap _map552 = iprot.readMapBegin();
+                struct.executor_node_port = new HashMap<List<Long>,NodeInfo>(2*_map552.size);
+                List<Long> _key553;
+                NodeInfo _val554;
+                for (int _i555 = 0; _i555 < _map552.size; ++_i555)
                 {
                   {
-                    org.apache.thrift.protocol.TList _list540 = iprot.readListBegin();
-                    _key537 = new ArrayList<Long>(_list540.size);
-                    long _elem541;
-                    for (int _i542 = 0; _i542 < _list540.size; ++_i542)
+                    org.apache.thrift.protocol.TList _list556 = iprot.readListBegin();
+                    _key553 = new ArrayList<Long>(_list556.size);
+                    long _elem557;
+                    for (int _i558 = 0; _i558 < _list556.size; ++_i558)
                     {
-                      _elem541 = iprot.readI64();
-                      _key537.add(_elem541);
+                      _elem557 = iprot.readI64();
+                      _key553.add(_elem557);
                     }
                     iprot.readListEnd();
                   }
-                  _val538 = new NodeInfo();
-                  _val538.read(iprot);
-                  struct.executor_node_port.put(_key537, _val538);
+                  _val554 = new NodeInfo();
+                  _val554.read(iprot);
+                  struct.executor_node_port.put(_key553, _val554);
                 }
                 iprot.readMapEnd();
               }
@@ -838,25 +838,25 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           case 4: // EXECUTOR_START_TIME_SECS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map543 = iprot.readMapBegin();
-                struct.executor_start_time_secs = new HashMap<List<Long>,Long>(2*_map543.size);
-                List<Long> _key544;
-                long _val545;
-                for (int _i546 = 0; _i546 < _map543.size; ++_i546)
+                org.apache.thrift.protocol.TMap _map559 = iprot.readMapBegin();
+                struct.executor_start_time_secs = new HashMap<List<Long>,Long>(2*_map559.size);
+                List<Long> _key560;
+                long _val561;
+                for (int _i562 = 0; _i562 < _map559.size; ++_i562)
                 {
                   {
-                    org.apache.thrift.protocol.TList _list547 = iprot.readListBegin();
-                    _key544 = new ArrayList<Long>(_list547.size);
-                    long _elem548;
-                    for (int _i549 = 0; _i549 < _list547.size; ++_i549)
+                    org.apache.thrift.protocol.TList _list563 = iprot.readListBegin();
+                    _key560 = new ArrayList<Long>(_list563.size);
+                    long _elem564;
+                    for (int _i565 = 0; _i565 < _list563.size; ++_i565)
                     {
-                      _elem548 = iprot.readI64();
-                      _key544.add(_elem548);
+                      _elem564 = iprot.readI64();
+                      _key560.add(_elem564);
                     }
                     iprot.readListEnd();
                   }
-                  _val545 = iprot.readI64();
-                  struct.executor_start_time_secs.put(_key544, _val545);
+                  _val561 = iprot.readI64();
+                  struct.executor_start_time_secs.put(_key560, _val561);
                 }
                 iprot.readMapEnd();
               }
@@ -868,17 +868,17 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           case 5: // WORKER_RESOURCES
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map550 = iprot.readMapBegin();
-                struct.worker_resources = new HashMap<NodeInfo,WorkerResources>(2*_map550.size);
-                NodeInfo _key551;
-                WorkerResources _val552;
-                for (int _i553 = 0; _i553 < _map550.size; ++_i553)
+                org.apache.thrift.protocol.TMap _map566 = iprot.readMapBegin();
+                struct.worker_resources = new HashMap<NodeInfo,WorkerResources>(2*_map566.size);
+                NodeInfo _key567;
+                WorkerResources _val568;
+                for (int _i569 = 0; _i569 < _map566.size; ++_i569)
                 {
-                  _key551 = new NodeInfo();
-                  _key551.read(iprot);
-                  _val552 = new WorkerResources();
-                  _val552.read(iprot);
-                  struct.worker_resources.put(_key551, _val552);
+                  _key567 = new NodeInfo();
+                  _key567.read(iprot);
+                  _val568 = new WorkerResources();
+                  _val568.read(iprot);
+                  struct.worker_resources.put(_key567, _val568);
                 }
                 iprot.readMapEnd();
               }
@@ -910,10 +910,10 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           oprot.writeFieldBegin(NODE_HOST_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.node_host.size()));
-            for (Map.Entry<String, String> _iter554 : struct.node_host.entrySet())
+            for (Map.Entry<String, String> _iter570 : struct.node_host.entrySet())
             {
-              oprot.writeString(_iter554.getKey());
-              oprot.writeString(_iter554.getValue());
+              oprot.writeString(_iter570.getKey());
+              oprot.writeString(_iter570.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -925,17 +925,17 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           oprot.writeFieldBegin(EXECUTOR_NODE_PORT_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.STRUCT, struct.executor_node_port.size()));
-            for (Map.Entry<List<Long>, NodeInfo> _iter555 : struct.executor_node_port.entrySet())
+            for (Map.Entry<List<Long>, NodeInfo> _iter571 : struct.executor_node_port.entrySet())
             {
               {
-                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter555.getKey().size()));
-                for (long _iter556 : _iter555.getKey())
+                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter571.getKey().size()));
+                for (long _iter572 : _iter571.getKey())
                 {
-                  oprot.writeI64(_iter556);
+                  oprot.writeI64(_iter572);
                 }
                 oprot.writeListEnd();
               }
-              _iter555.getValue().write(oprot);
+              _iter571.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -947,17 +947,17 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           oprot.writeFieldBegin(EXECUTOR_START_TIME_SECS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.I64, struct.executor_start_time_secs.size()));
-            for (Map.Entry<List<Long>, Long> _iter557 : struct.executor_start_time_secs.entrySet())
+            for (Map.Entry<List<Long>, Long> _iter573 : struct.executor_start_time_secs.entrySet())
             {
               {
-                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter557.getKey().size()));
-                for (long _iter558 : _iter557.getKey())
+                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter573.getKey().size()));
+                for (long _iter574 : _iter573.getKey())
                 {
-                  oprot.writeI64(_iter558);
+                  oprot.writeI64(_iter574);
                 }
                 oprot.writeListEnd();
               }
-              oprot.writeI64(_iter557.getValue());
+              oprot.writeI64(_iter573.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -969,10 +969,10 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           oprot.writeFieldBegin(WORKER_RESOURCES_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, struct.worker_resources.size()));
-            for (Map.Entry<NodeInfo, WorkerResources> _iter559 : struct.worker_resources.entrySet())
+            for (Map.Entry<NodeInfo, WorkerResources> _iter575 : struct.worker_resources.entrySet())
             {
-              _iter559.getKey().write(oprot);
-              _iter559.getValue().write(oprot);
+              _iter575.getKey().write(oprot);
+              _iter575.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -1014,52 +1014,52 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
       if (struct.is_set_node_host()) {
         {
           oprot.writeI32(struct.node_host.size());
-          for (Map.Entry<String, String> _iter560 : struct.node_host.entrySet())
+          for (Map.Entry<String, String> _iter576 : struct.node_host.entrySet())
           {
-            oprot.writeString(_iter560.getKey());
-            oprot.writeString(_iter560.getValue());
+            oprot.writeString(_iter576.getKey());
+            oprot.writeString(_iter576.getValue());
           }
         }
       }
       if (struct.is_set_executor_node_port()) {
         {
           oprot.writeI32(struct.executor_node_port.size());
-          for (Map.Entry<List<Long>, NodeInfo> _iter561 : struct.executor_node_port.entrySet())
+          for (Map.Entry<List<Long>, NodeInfo> _iter577 : struct.executor_node_port.entrySet())
           {
             {
-              oprot.writeI32(_iter561.getKey().size());
-              for (long _iter562 : _iter561.getKey())
+              oprot.writeI32(_iter577.getKey().size());
+              for (long _iter578 : _iter577.getKey())
               {
-                oprot.writeI64(_iter562);
+                oprot.writeI64(_iter578);
               }
             }
-            _iter561.getValue().write(oprot);
+            _iter577.getValue().write(oprot);
           }
         }
       }
       if (struct.is_set_executor_start_time_secs()) {
         {
           oprot.writeI32(struct.executor_start_time_secs.size());
-          for (Map.Entry<List<Long>, Long> _iter563 : struct.executor_start_time_secs.entrySet())
+          for (Map.Entry<List<Long>, Long> _iter579 : struct.executor_start_time_secs.entrySet())
           {
             {
-              oprot.writeI32(_iter563.getKey().size());
-              for (long _iter564 : _iter563.getKey())
+              oprot.writeI32(_iter579.getKey().size());
+              for (long _iter580 : _iter579.getKey())
               {
-                oprot.writeI64(_iter564);
+                oprot.writeI64(_iter580);
               }
             }
-            oprot.writeI64(_iter563.getValue());
+            oprot.writeI64(_iter579.getValue());
           }
         }
       }
       if (struct.is_set_worker_resources()) {
         {
           oprot.writeI32(struct.worker_resources.size());
-          for (Map.Entry<NodeInfo, WorkerResources> _iter565 : struct.worker_resources.entrySet())
+          for (Map.Entry<NodeInfo, WorkerResources> _iter581 : struct.worker_resources.entrySet())
           {
-            _iter565.getKey().write(oprot);
-            _iter565.getValue().write(oprot);
+            _iter581.getKey().write(oprot);
+            _iter581.getValue().write(oprot);
           }
         }
       }
@@ -1073,81 +1073,81 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
       BitSet incoming = iprot.readBitSet(4);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TMap _map566 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.node_host = new HashMap<String,String>(2*_map566.size);
-          String _key567;
-          String _val568;
-          for (int _i569 = 0; _i569 < _map566.size; ++_i569)
+          org.apache.thrift.protocol.TMap _map582 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.node_host = new HashMap<String,String>(2*_map582.size);
+          String _key583;
+          String _val584;
+          for (int _i585 = 0; _i585 < _map582.size; ++_i585)
           {
-            _key567 = iprot.readString();
-            _val568 = iprot.readString();
-            struct.node_host.put(_key567, _val568);
+            _key583 = iprot.readString();
+            _val584 = iprot.readString();
+            struct.node_host.put(_key583, _val584);
           }
         }
         struct.set_node_host_isSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TMap _map570 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.executor_node_port = new HashMap<List<Long>,NodeInfo>(2*_map570.size);
-          List<Long> _key571;
-          NodeInfo _val572;
-          for (int _i573 = 0; _i573 < _map570.size; ++_i573)
+          org.apache.thrift.protocol.TMap _map586 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.executor_node_port = new HashMap<List<Long>,NodeInfo>(2*_map586.size);
+          List<Long> _key587;
+          NodeInfo _val588;
+          for (int _i589 = 0; _i589 < _map586.size; ++_i589)
           {
             {
-              org.apache.thrift.protocol.TList _list574 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-              _key571 = new ArrayList<Long>(_list574.size);
-              long _elem575;
-              for (int _i576 = 0; _i576 < _list574.size; ++_i576)
+              org.apache.thrift.protocol.TList _list590 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+              _key587 = new ArrayList<Long>(_list590.size);
+              long _elem591;
+              for (int _i592 = 0; _i592 < _list590.size; ++_i592)
               {
-                _elem575 = iprot.readI64();
-                _key571.add(_elem575);
+                _elem591 = iprot.readI64();
+                _key587.add(_elem591);
               }
             }
-            _val572 = new NodeInfo();
-            _val572.read(iprot);
-            struct.executor_node_port.put(_key571, _val572);
+            _val588 = new NodeInfo();
+            _val588.read(iprot);
+            struct.executor_node_port.put(_key587, _val588);
           }
         }
         struct.set_executor_node_port_isSet(true);
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TMap _map577 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.I64, iprot.readI32());
-          struct.executor_start_time_secs = new HashMap<List<Long>,Long>(2*_map577.size);
-          List<Long> _key578;
-          long _val579;
-          for (int _i580 = 0; _i580 < _map577.size; ++_i580)
+          org.apache.thrift.protocol.TMap _map593 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.executor_start_time_secs = new HashMap<List<Long>,Long>(2*_map593.size);
+          List<Long> _key594;
+          long _val595;
+          for (int _i596 = 0; _i596 < _map593.size; ++_i596)
           {
             {
-              org.apache.thrift.protocol.TList _list581 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-              _key578 = new ArrayList<Long>(_list581.size);
-              long _elem582;
-              for (int _i583 = 0; _i583 < _list581.size; ++_i583)
+              org.apache.thrift.protocol.TList _list597 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+              _key594 = new ArrayList<Long>(_list597.size);
+              long _elem598;
+              for (int _i599 = 0; _i599 < _list597.size; ++_i599)
               {
-                _elem582 = iprot.readI64();
-                _key578.add(_elem582);
+                _elem598 = iprot.readI64();
+                _key594.add(_elem598);
               }
             }
-            _val579 = iprot.readI64();
-            struct.executor_start_time_secs.put(_key578, _val579);
+            _val595 = iprot.readI64();
+            struct.executor_start_time_secs.put(_key594, _val595);
           }
         }
         struct.set_executor_start_time_secs_isSet(true);
       }
       if (incoming.get(3)) {
         {
-          org.apache.thrift.protocol.TMap _map584 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.worker_resources = new HashMap<NodeInfo,WorkerResources>(2*_map584.size);
-          NodeInfo _key585;
-          WorkerResources _val586;
-          for (int _i587 = 0; _i587 < _map584.size; ++_i587)
+          org.apache.thrift.protocol.TMap _map600 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.worker_resources = new HashMap<NodeInfo,WorkerResources>(2*_map600.size);
+          NodeInfo _key601;
+          WorkerResources _val602;
+          for (int _i603 = 0; _i603 < _map600.size; ++_i603)
           {
-            _key585 = new NodeInfo();
-            _key585.read(iprot);
-            _val586 = new WorkerResources();
-            _val586.read(iprot);
-            struct.worker_resources.put(_key585, _val586);
+            _key601 = new NodeInfo();
+            _key601.read(iprot);
+            _val602 = new WorkerResources();
+            _val602.read(iprot);
+            struct.worker_resources.put(_key601, _val602);
           }
         }
         struct.set_worker_resources_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/generated/BeginDownloadResult.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/BeginDownloadResult.java b/storm-core/src/jvm/backtype/storm/generated/BeginDownloadResult.java
new file mode 100644
index 0000000..f01c4eb
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/BeginDownloadResult.java
@@ -0,0 +1,608 @@
+/**
+ * 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.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class BeginDownloadResult implements org.apache.thrift.TBase<BeginDownloadResult, BeginDownloadResult._Fields>, java.io.Serializable, Cloneable, Comparable<BeginDownloadResult> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("BeginDownloadResult");
+
+  private static final org.apache.thrift.protocol.TField VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("version", org.apache.thrift.protocol.TType.I64, (short)1);
+  private static final org.apache.thrift.protocol.TField SESSION_FIELD_DESC = new org.apache.thrift.protocol.TField("session", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField DATA_SIZE_FIELD_DESC = new org.apache.thrift.protocol.TField("data_size", org.apache.thrift.protocol.TType.I64, (short)3);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new BeginDownloadResultStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new BeginDownloadResultTupleSchemeFactory());
+  }
+
+  private long version; // required
+  private String session; // required
+  private long data_size; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    VERSION((short)1, "version"),
+    SESSION((short)2, "session"),
+    DATA_SIZE((short)3, "data_size");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // VERSION
+          return VERSION;
+        case 2: // SESSION
+          return SESSION;
+        case 3: // DATA_SIZE
+          return DATA_SIZE;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __VERSION_ISSET_ID = 0;
+  private static final int __DATA_SIZE_ISSET_ID = 1;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.DATA_SIZE};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.VERSION, new org.apache.thrift.meta_data.FieldMetaData("version", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.SESSION, new org.apache.thrift.meta_data.FieldMetaData("session", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.DATA_SIZE, new org.apache.thrift.meta_data.FieldMetaData("data_size", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(BeginDownloadResult.class, metaDataMap);
+  }
+
+  public BeginDownloadResult() {
+  }
+
+  public BeginDownloadResult(
+    long version,
+    String session)
+  {
+    this();
+    this.version = version;
+    set_version_isSet(true);
+    this.session = session;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public BeginDownloadResult(BeginDownloadResult other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.version = other.version;
+    if (other.is_set_session()) {
+      this.session = other.session;
+    }
+    this.data_size = other.data_size;
+  }
+
+  public BeginDownloadResult deepCopy() {
+    return new BeginDownloadResult(this);
+  }
+
+  @Override
+  public void clear() {
+    set_version_isSet(false);
+    this.version = 0;
+    this.session = null;
+    set_data_size_isSet(false);
+    this.data_size = 0;
+  }
+
+  public long get_version() {
+    return this.version;
+  }
+
+  public void set_version(long version) {
+    this.version = version;
+    set_version_isSet(true);
+  }
+
+  public void unset_version() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VERSION_ISSET_ID);
+  }
+
+  /** Returns true if field version is set (has been assigned a value) and false otherwise */
+  public boolean is_set_version() {
+    return EncodingUtils.testBit(__isset_bitfield, __VERSION_ISSET_ID);
+  }
+
+  public void set_version_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VERSION_ISSET_ID, value);
+  }
+
+  public String get_session() {
+    return this.session;
+  }
+
+  public void set_session(String session) {
+    this.session = session;
+  }
+
+  public void unset_session() {
+    this.session = null;
+  }
+
+  /** Returns true if field session is set (has been assigned a value) and false otherwise */
+  public boolean is_set_session() {
+    return this.session != null;
+  }
+
+  public void set_session_isSet(boolean value) {
+    if (!value) {
+      this.session = null;
+    }
+  }
+
+  public long get_data_size() {
+    return this.data_size;
+  }
+
+  public void set_data_size(long data_size) {
+    this.data_size = data_size;
+    set_data_size_isSet(true);
+  }
+
+  public void unset_data_size() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __DATA_SIZE_ISSET_ID);
+  }
+
+  /** Returns true if field data_size is set (has been assigned a value) and false otherwise */
+  public boolean is_set_data_size() {
+    return EncodingUtils.testBit(__isset_bitfield, __DATA_SIZE_ISSET_ID);
+  }
+
+  public void set_data_size_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __DATA_SIZE_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case VERSION:
+      if (value == null) {
+        unset_version();
+      } else {
+        set_version((Long)value);
+      }
+      break;
+
+    case SESSION:
+      if (value == null) {
+        unset_session();
+      } else {
+        set_session((String)value);
+      }
+      break;
+
+    case DATA_SIZE:
+      if (value == null) {
+        unset_data_size();
+      } else {
+        set_data_size((Long)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case VERSION:
+      return get_version();
+
+    case SESSION:
+      return get_session();
+
+    case DATA_SIZE:
+      return get_data_size();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case VERSION:
+      return is_set_version();
+    case SESSION:
+      return is_set_session();
+    case DATA_SIZE:
+      return is_set_data_size();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof BeginDownloadResult)
+      return this.equals((BeginDownloadResult)that);
+    return false;
+  }
+
+  public boolean equals(BeginDownloadResult that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_version = true;
+    boolean that_present_version = true;
+    if (this_present_version || that_present_version) {
+      if (!(this_present_version && that_present_version))
+        return false;
+      if (this.version != that.version)
+        return false;
+    }
+
+    boolean this_present_session = true && this.is_set_session();
+    boolean that_present_session = true && that.is_set_session();
+    if (this_present_session || that_present_session) {
+      if (!(this_present_session && that_present_session))
+        return false;
+      if (!this.session.equals(that.session))
+        return false;
+    }
+
+    boolean this_present_data_size = true && this.is_set_data_size();
+    boolean that_present_data_size = true && that.is_set_data_size();
+    if (this_present_data_size || that_present_data_size) {
+      if (!(this_present_data_size && that_present_data_size))
+        return false;
+      if (this.data_size != that.data_size)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_version = true;
+    list.add(present_version);
+    if (present_version)
+      list.add(version);
+
+    boolean present_session = true && (is_set_session());
+    list.add(present_session);
+    if (present_session)
+      list.add(session);
+
+    boolean present_data_size = true && (is_set_data_size());
+    list.add(present_data_size);
+    if (present_data_size)
+      list.add(data_size);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(BeginDownloadResult other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_version()).compareTo(other.is_set_version());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_version()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.version, other.version);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_session()).compareTo(other.is_set_session());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_session()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.session, other.session);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_data_size()).compareTo(other.is_set_data_size());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_data_size()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.data_size, other.data_size);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("BeginDownloadResult(");
+    boolean first = true;
+
+    sb.append("version:");
+    sb.append(this.version);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("session:");
+    if (this.session == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.session);
+    }
+    first = false;
+    if (is_set_data_size()) {
+      if (!first) sb.append(", ");
+      sb.append("data_size:");
+      sb.append(this.data_size);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_version()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'version' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_session()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'session' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class BeginDownloadResultStandardSchemeFactory implements SchemeFactory {
+    public BeginDownloadResultStandardScheme getScheme() {
+      return new BeginDownloadResultStandardScheme();
+    }
+  }
+
+  private static class BeginDownloadResultStandardScheme extends StandardScheme<BeginDownloadResult> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, BeginDownloadResult struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // VERSION
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.version = iprot.readI64();
+              struct.set_version_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // SESSION
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.session = iprot.readString();
+              struct.set_session_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // DATA_SIZE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.data_size = iprot.readI64();
+              struct.set_data_size_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, BeginDownloadResult struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldBegin(VERSION_FIELD_DESC);
+      oprot.writeI64(struct.version);
+      oprot.writeFieldEnd();
+      if (struct.session != null) {
+        oprot.writeFieldBegin(SESSION_FIELD_DESC);
+        oprot.writeString(struct.session);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_data_size()) {
+        oprot.writeFieldBegin(DATA_SIZE_FIELD_DESC);
+        oprot.writeI64(struct.data_size);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class BeginDownloadResultTupleSchemeFactory implements SchemeFactory {
+    public BeginDownloadResultTupleScheme getScheme() {
+      return new BeginDownloadResultTupleScheme();
+    }
+  }
+
+  private static class BeginDownloadResultTupleScheme extends TupleScheme<BeginDownloadResult> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, BeginDownloadResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeI64(struct.version);
+      oprot.writeString(struct.session);
+      BitSet optionals = new BitSet();
+      if (struct.is_set_data_size()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_data_size()) {
+        oprot.writeI64(struct.data_size);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, BeginDownloadResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.version = iprot.readI64();
+      struct.set_version_isSet(true);
+      struct.session = iprot.readString();
+      struct.set_session_isSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.data_size = iprot.readI64();
+        struct.set_data_size_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java b/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java
index a1b7e2e..2754abd 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java
@@ -635,17 +635,17 @@ public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWo
           case 2: // EXECUTOR_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map608 = iprot.readMapBegin();
-                struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map608.size);
-                ExecutorInfo _key609;
-                ExecutorStats _val610;
-                for (int _i611 = 0; _i611 < _map608.size; ++_i611)
+                org.apache.thrift.protocol.TMap _map624 = iprot.readMapBegin();
+                struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map624.size);
+                ExecutorInfo _key625;
+                ExecutorStats _val626;
+                for (int _i627 = 0; _i627 < _map624.size; ++_i627)
                 {
-                  _key609 = new ExecutorInfo();
-                  _key609.read(iprot);
-                  _val610 = new ExecutorStats();
-                  _val610.read(iprot);
-                  struct.executor_stats.put(_key609, _val610);
+                  _key625 = new ExecutorInfo();
+                  _key625.read(iprot);
+                  _val626 = new ExecutorStats();
+                  _val626.read(iprot);
+                  struct.executor_stats.put(_key625, _val626);
                 }
                 iprot.readMapEnd();
               }
@@ -692,10 +692,10 @@ public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWo
         oprot.writeFieldBegin(EXECUTOR_STATS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, struct.executor_stats.size()));
-          for (Map.Entry<ExecutorInfo, ExecutorStats> _iter612 : struct.executor_stats.entrySet())
+          for (Map.Entry<ExecutorInfo, ExecutorStats> _iter628 : struct.executor_stats.entrySet())
           {
-            _iter612.getKey().write(oprot);
-            _iter612.getValue().write(oprot);
+            _iter628.getKey().write(oprot);
+            _iter628.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -727,10 +727,10 @@ public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWo
       oprot.writeString(struct.storm_id);
       {
         oprot.writeI32(struct.executor_stats.size());
-        for (Map.Entry<ExecutorInfo, ExecutorStats> _iter613 : struct.executor_stats.entrySet())
+        for (Map.Entry<ExecutorInfo, ExecutorStats> _iter629 : struct.executor_stats.entrySet())
         {
-          _iter613.getKey().write(oprot);
-          _iter613.getValue().write(oprot);
+          _iter629.getKey().write(oprot);
+          _iter629.getValue().write(oprot);
         }
       }
       oprot.writeI32(struct.time_secs);
@@ -743,17 +743,17 @@ public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWo
       struct.storm_id = iprot.readString();
       struct.set_storm_id_isSet(true);
       {
-        org.apache.thrift.protocol.TMap _map614 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map614.size);
-        ExecutorInfo _key615;
-        ExecutorStats _val616;
-        for (int _i617 = 0; _i617 < _map614.size; ++_i617)
+        org.apache.thrift.protocol.TMap _map630 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map630.size);
+        ExecutorInfo _key631;
+        ExecutorStats _val632;
+        for (int _i633 = 0; _i633 < _map630.size; ++_i633)
         {
-          _key615 = new ExecutorInfo();
-          _key615.read(iprot);
-          _val616 = new ExecutorStats();
-          _val616.read(iprot);
-          struct.executor_stats.put(_key615, _val616);
+          _key631 = new ExecutorInfo();
+          _key631.read(iprot);
+          _val632 = new ExecutorStats();
+          _val632.read(iprot);
+          struct.executor_stats.put(_key631, _val632);
         }
       }
       struct.set_executor_stats_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/generated/HBNodes.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/HBNodes.java b/storm-core/src/jvm/backtype/storm/generated/HBNodes.java
index b6fc526..84bac76 100644
--- a/storm-core/src/jvm/backtype/storm/generated/HBNodes.java
+++ b/storm-core/src/jvm/backtype/storm/generated/HBNodes.java
@@ -364,13 +364,13 @@ public class HBNodes implements org.apache.thrift.TBase<HBNodes, HBNodes._Fields
           case 1: // PULSE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list706 = iprot.readListBegin();
-                struct.pulseIds = new ArrayList<String>(_list706.size);
-                String _elem707;
-                for (int _i708 = 0; _i708 < _list706.size; ++_i708)
+                org.apache.thrift.protocol.TList _list730 = iprot.readListBegin();
+                struct.pulseIds = new ArrayList<String>(_list730.size);
+                String _elem731;
+                for (int _i732 = 0; _i732 < _list730.size; ++_i732)
                 {
-                  _elem707 = iprot.readString();
-                  struct.pulseIds.add(_elem707);
+                  _elem731 = iprot.readString();
+                  struct.pulseIds.add(_elem731);
                 }
                 iprot.readListEnd();
               }
@@ -396,9 +396,9 @@ public class HBNodes implements org.apache.thrift.TBase<HBNodes, HBNodes._Fields
         oprot.writeFieldBegin(PULSE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.pulseIds.size()));
-          for (String _iter709 : struct.pulseIds)
+          for (String _iter733 : struct.pulseIds)
           {
-            oprot.writeString(_iter709);
+            oprot.writeString(_iter733);
           }
           oprot.writeListEnd();
         }
@@ -429,9 +429,9 @@ public class HBNodes implements org.apache.thrift.TBase<HBNodes, HBNodes._Fields
       if (struct.is_set_pulseIds()) {
         {
           oprot.writeI32(struct.pulseIds.size());
-          for (String _iter710 : struct.pulseIds)
+          for (String _iter734 : struct.pulseIds)
           {
-            oprot.writeString(_iter710);
+            oprot.writeString(_iter734);
           }
         }
       }
@@ -443,13 +443,13 @@ public class HBNodes implements org.apache.thrift.TBase<HBNodes, HBNodes._Fields
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list711 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.pulseIds = new ArrayList<String>(_list711.size);
-          String _elem712;
-          for (int _i713 = 0; _i713 < _list711.size; ++_i713)
+          org.apache.thrift.protocol.TList _list735 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.pulseIds = new ArrayList<String>(_list735.size);
+          String _elem736;
+          for (int _i737 = 0; _i737 < _list735.size; ++_i737)
           {
-            _elem712 = iprot.readString();
-            struct.pulseIds.add(_elem712);
+            _elem736 = iprot.readString();
+            struct.pulseIds.add(_elem736);
           }
         }
         struct.set_pulseIds_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/generated/HBRecords.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/HBRecords.java b/storm-core/src/jvm/backtype/storm/generated/HBRecords.java
index 4767068..65de1cc 100644
--- a/storm-core/src/jvm/backtype/storm/generated/HBRecords.java
+++ b/storm-core/src/jvm/backtype/storm/generated/HBRecords.java
@@ -367,14 +367,14 @@ public class HBRecords implements org.apache.thrift.TBase<HBRecords, HBRecords._
           case 1: // PULSES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list698 = iprot.readListBegin();
-                struct.pulses = new ArrayList<HBPulse>(_list698.size);
-                HBPulse _elem699;
-                for (int _i700 = 0; _i700 < _list698.size; ++_i700)
+                org.apache.thrift.protocol.TList _list722 = iprot.readListBegin();
+                struct.pulses = new ArrayList<HBPulse>(_list722.size);
+                HBPulse _elem723;
+                for (int _i724 = 0; _i724 < _list722.size; ++_i724)
                 {
-                  _elem699 = new HBPulse();
-                  _elem699.read(iprot);
-                  struct.pulses.add(_elem699);
+                  _elem723 = new HBPulse();
+                  _elem723.read(iprot);
+                  struct.pulses.add(_elem723);
                 }
                 iprot.readListEnd();
               }
@@ -400,9 +400,9 @@ public class HBRecords implements org.apache.thrift.TBase<HBRecords, HBRecords._
         oprot.writeFieldBegin(PULSES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.pulses.size()));
-          for (HBPulse _iter701 : struct.pulses)
+          for (HBPulse _iter725 : struct.pulses)
           {
-            _iter701.write(oprot);
+            _iter725.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -433,9 +433,9 @@ public class HBRecords implements org.apache.thrift.TBase<HBRecords, HBRecords._
       if (struct.is_set_pulses()) {
         {
           oprot.writeI32(struct.pulses.size());
-          for (HBPulse _iter702 : struct.pulses)
+          for (HBPulse _iter726 : struct.pulses)
           {
-            _iter702.write(oprot);
+            _iter726.write(oprot);
           }
         }
       }
@@ -447,14 +447,14 @@ public class HBRecords implements org.apache.thrift.TBase<HBRecords, HBRecords._
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list703 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.pulses = new ArrayList<HBPulse>(_list703.size);
-          HBPulse _elem704;
-          for (int _i705 = 0; _i705 < _list703.size; ++_i705)
+          org.apache.thrift.protocol.TList _list727 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.pulses = new ArrayList<HBPulse>(_list727.size);
+          HBPulse _elem728;
+          for (int _i729 = 0; _i729 < _list727.size; ++_i729)
           {
-            _elem704 = new HBPulse();
-            _elem704.read(iprot);
-            struct.pulses.add(_elem704);
+            _elem728 = new HBPulse();
+            _elem728.read(iprot);
+            struct.pulses.add(_elem728);
           }
         }
         struct.set_pulses_isSet(true);


[16/17] storm git commit: Merge branch 'blobstore' of https://github.com/redsanket/storm into STORM-876

Posted by bo...@apache.org.
Merge branch 'blobstore' of https://github.com/redsanket/storm into STORM-876

STORM-876: Blobstore/DistCache Support


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

Branch: refs/heads/master
Commit: bdbec852a16a758074079eb0bce70ac77c876a5f
Parents: ce2d49b 5105802
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Fri Dec 4 08:54:07 2015 -0600
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Fri Dec 4 08:54:07 2015 -0600

----------------------------------------------------------------------
 bin/storm.py                                    |    28 +-
 conf/defaults.yaml                              |    16 +-
 external/storm-hdfs/pom.xml                     |    70 +
 .../storm/hdfs/blobstore/HdfsBlobStore.java     |   384 +
 .../storm/hdfs/blobstore/HdfsBlobStoreFile.java |   196 +
 .../storm/hdfs/blobstore/HdfsBlobStoreImpl.java |   312 +
 .../hdfs/blobstore/HdfsClientBlobStore.java     |   120 +
 .../ha/codedistributor/HDFSCodeDistributor.java |   118 -
 .../storm/hdfs/blobstore/BlobStoreTest.java     |   530 +
 .../hdfs/blobstore/HdfsBlobStoreImplTest.java   |   231 +
 pom.xml                                         |    16 +-
 storm-core/pom.xml                              |    36 +-
 storm-core/src/clj/backtype/storm/blobstore.clj |    28 +
 storm-core/src/clj/backtype/storm/cluster.clj   |    92 +-
 .../cluster_state/zookeeper_state_factory.clj   |     4 +
 .../clj/backtype/storm/command/blobstore.clj    |   162 +
 storm-core/src/clj/backtype/storm/config.clj    |    24 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |   676 +-
 .../clj/backtype/storm/daemon/supervisor.clj    |   479 +-
 storm-core/src/clj/backtype/storm/testing.clj   |     5 +-
 storm-core/src/clj/backtype/storm/util.clj      |    16 +-
 storm-core/src/clj/backtype/storm/zookeeper.clj |    31 +-
 storm-core/src/jvm/backtype/storm/Config.java   |   123 +-
 .../storm/blobstore/AtomicOutputStream.java     |    32 +
 .../storm/blobstore/BlobKeySequenceInfo.java    |    40 +
 .../jvm/backtype/storm/blobstore/BlobStore.java |   445 +
 .../storm/blobstore/BlobStoreAclHandler.java    |   399 +
 .../backtype/storm/blobstore/BlobStoreFile.java |    50 +
 .../storm/blobstore/BlobStoreUtils.java         |   257 +
 .../storm/blobstore/BlobSynchronizer.java       |   124 +
 .../storm/blobstore/ClientBlobStore.java        |   184 +
 .../storm/blobstore/FileBlobStoreImpl.java      |   248 +
 .../storm/blobstore/InputStreamWithMeta.java    |    26 +
 .../jvm/backtype/storm/blobstore/KeyFilter.java |    22 +
 .../storm/blobstore/KeySequenceNumber.java      |   229 +
 .../storm/blobstore/LocalFsBlobStore.java       |   323 +
 .../storm/blobstore/LocalFsBlobStoreFile.java   |   159 +
 .../storm/blobstore/NimbusBlobStore.java        |   420 +
 .../backtype/storm/cluster/ClusterState.java    |     9 +
 .../storm/codedistributor/ICodeDistributor.java |    73 -
 .../LocalFileSystemCodeDistributor.java         |   123 -
 .../backtype/storm/generated/AccessControl.java |   627 +
 .../storm/generated/AccessControlType.java      |    62 +
 .../backtype/storm/generated/Assignment.java    |   244 +-
 .../storm/generated/BeginDownloadResult.java    |   608 +
 .../storm/generated/ClusterWorkerHeartbeat.java |    52 +-
 .../jvm/backtype/storm/generated/HBNodes.java   |    32 +-
 .../jvm/backtype/storm/generated/HBRecords.java |    36 +-
 .../generated/KeyAlreadyExistsException.java    |   406 +
 .../storm/generated/KeyNotFoundException.java   |   406 +
 .../storm/generated/LSApprovedWorkers.java      |    44 +-
 .../generated/LSSupervisorAssignments.java      |    48 +-
 .../backtype/storm/generated/LSTopoHistory.java |    64 +-
 .../storm/generated/LSTopoHistoryList.java      |    36 +-
 .../storm/generated/LSWorkerHeartbeat.java      |    36 +-
 .../storm/generated/ListBlobsResult.java        |   556 +
 .../storm/generated/LocalAssignment.java        |    36 +-
 .../storm/generated/LocalStateData.java         |    48 +-
 .../jvm/backtype/storm/generated/LogConfig.java |    48 +-
 .../jvm/backtype/storm/generated/Nimbus.java    | 26917 +++++++++++++----
 .../jvm/backtype/storm/generated/NodeInfo.java  |    32 +-
 .../storm/generated/ReadableBlobMeta.java       |   510 +
 .../storm/generated/SettableBlobMeta.java       |   567 +
 .../jvm/backtype/storm/generated/StormBase.java |    92 +-
 .../storm/generated/SupervisorInfo.java         |   152 +-
 .../storm/generated/TopologyHistoryInfo.java    |    32 +-
 .../backtype/storm/localizer/LocalResource.java |    44 +
 .../storm/localizer/LocalizedResource.java      |   130 +
 .../LocalizedResourceRetentionSet.java          |   140 +
 .../storm/localizer/LocalizedResourceSet.java   |   101 +
 .../jvm/backtype/storm/localizer/Localizer.java |   695 +
 .../storm/security/auth/NimbusPrincipal.java    |    29 +
 .../backtype/storm/utils/BufferInputStream.java |    53 +
 .../jvm/backtype/storm/utils/ShellUtils.java    |     7 +
 .../src/jvm/backtype/storm/utils/Utils.java     |   545 +-
 .../storm/validation/ConfigValidation.java      |    16 +-
 .../validation/ConfigValidationAnnotations.java |    11 +-
 storm-core/src/py/storm/Nimbus-remote           |    98 +
 storm-core/src/py/storm/Nimbus.py               |  5991 +++-
 storm-core/src/py/storm/ttypes.py               |   996 +-
 storm-core/src/storm.thrift                     |    59 +
 .../test/clj/backtype/storm/cluster_test.clj    |    20 +-
 .../test/clj/backtype/storm/nimbus_test.clj     |    43 +-
 .../storm/security/auth/ReqContext_test.clj     |     1 +
 .../test/clj/backtype/storm/supervisor_test.clj |    18 +-
 .../backtype/storm/blobstore/BlobStoreTest.java |   461 +
 .../storm/blobstore/BlobSynchronizerTest.java   |   137 +
 .../storm/blobstore/ClientBlobStoreTest.java    |   179 +
 .../LocalizedResourceRetentionSetTest.java      |    85 +
 .../localizer/LocalizedResourceSetTest.java     |    74 +
 .../backtype/storm/localizer/LocalizerTest.java |   671 +
 .../jvm/backtype/storm/localizer/localtest.zip  |   Bin 0 -> 6378 bytes
 .../storm/localizer/localtestwithsymlink.jar    |   Bin 0 -> 6591 bytes
 .../storm/localizer/localtestwithsymlink.tar    |   Bin 0 -> 24576 bytes
 .../storm/localizer/localtestwithsymlink.tar.gz |   Bin 0 -> 6106 bytes
 .../storm/localizer/localtestwithsymlink.tgz    |   Bin 0 -> 6106 bytes
 96 files changed, 39640 insertions(+), 9515 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/bdbec852/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/bdbec852/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/bdbec852/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/bdbec852/storm-core/src/jvm/backtype/storm/validation/ConfigValidationAnnotations.java
----------------------------------------------------------------------


[07/17] storm git commit: Blobstore API STORM- 876

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/generated/KeyAlreadyExistsException.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/KeyAlreadyExistsException.java b/storm-core/src/jvm/backtype/storm/generated/KeyAlreadyExistsException.java
new file mode 100644
index 0000000..fe1a8d0
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/KeyAlreadyExistsException.java
@@ -0,0 +1,406 @@
+/**
+ * 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.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class KeyAlreadyExistsException extends TException implements org.apache.thrift.TBase<KeyAlreadyExistsException, KeyAlreadyExistsException._Fields>, java.io.Serializable, Cloneable, Comparable<KeyAlreadyExistsException> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("KeyAlreadyExistsException");
+
+  private static final org.apache.thrift.protocol.TField MSG_FIELD_DESC = new org.apache.thrift.protocol.TField("msg", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new KeyAlreadyExistsExceptionStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new KeyAlreadyExistsExceptionTupleSchemeFactory());
+  }
+
+  private String msg; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    MSG((short)1, "msg");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // MSG
+          return MSG;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.MSG, new org.apache.thrift.meta_data.FieldMetaData("msg", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(KeyAlreadyExistsException.class, metaDataMap);
+  }
+
+  public KeyAlreadyExistsException() {
+  }
+
+  public KeyAlreadyExistsException(
+    String msg)
+  {
+    this();
+    this.msg = msg;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public KeyAlreadyExistsException(KeyAlreadyExistsException other) {
+    if (other.is_set_msg()) {
+      this.msg = other.msg;
+    }
+  }
+
+  public KeyAlreadyExistsException deepCopy() {
+    return new KeyAlreadyExistsException(this);
+  }
+
+  @Override
+  public void clear() {
+    this.msg = null;
+  }
+
+  public String get_msg() {
+    return this.msg;
+  }
+
+  public void set_msg(String msg) {
+    this.msg = msg;
+  }
+
+  public void unset_msg() {
+    this.msg = null;
+  }
+
+  /** Returns true if field msg is set (has been assigned a value) and false otherwise */
+  public boolean is_set_msg() {
+    return this.msg != null;
+  }
+
+  public void set_msg_isSet(boolean value) {
+    if (!value) {
+      this.msg = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case MSG:
+      if (value == null) {
+        unset_msg();
+      } else {
+        set_msg((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case MSG:
+      return get_msg();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case MSG:
+      return is_set_msg();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof KeyAlreadyExistsException)
+      return this.equals((KeyAlreadyExistsException)that);
+    return false;
+  }
+
+  public boolean equals(KeyAlreadyExistsException that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_msg = true && this.is_set_msg();
+    boolean that_present_msg = true && that.is_set_msg();
+    if (this_present_msg || that_present_msg) {
+      if (!(this_present_msg && that_present_msg))
+        return false;
+      if (!this.msg.equals(that.msg))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_msg = true && (is_set_msg());
+    list.add(present_msg);
+    if (present_msg)
+      list.add(msg);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(KeyAlreadyExistsException other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_msg()).compareTo(other.is_set_msg());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_msg()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.msg, other.msg);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("KeyAlreadyExistsException(");
+    boolean first = true;
+
+    sb.append("msg:");
+    if (this.msg == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.msg);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_msg()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class KeyAlreadyExistsExceptionStandardSchemeFactory implements SchemeFactory {
+    public KeyAlreadyExistsExceptionStandardScheme getScheme() {
+      return new KeyAlreadyExistsExceptionStandardScheme();
+    }
+  }
+
+  private static class KeyAlreadyExistsExceptionStandardScheme extends StandardScheme<KeyAlreadyExistsException> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, KeyAlreadyExistsException struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // MSG
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.msg = iprot.readString();
+              struct.set_msg_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, KeyAlreadyExistsException struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.msg != null) {
+        oprot.writeFieldBegin(MSG_FIELD_DESC);
+        oprot.writeString(struct.msg);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class KeyAlreadyExistsExceptionTupleSchemeFactory implements SchemeFactory {
+    public KeyAlreadyExistsExceptionTupleScheme getScheme() {
+      return new KeyAlreadyExistsExceptionTupleScheme();
+    }
+  }
+
+  private static class KeyAlreadyExistsExceptionTupleScheme extends TupleScheme<KeyAlreadyExistsException> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, KeyAlreadyExistsException struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.msg);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, KeyAlreadyExistsException struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.msg = iprot.readString();
+      struct.set_msg_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/generated/KeyNotFoundException.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/KeyNotFoundException.java b/storm-core/src/jvm/backtype/storm/generated/KeyNotFoundException.java
new file mode 100644
index 0000000..d3d6ee7
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/KeyNotFoundException.java
@@ -0,0 +1,406 @@
+/**
+ * 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.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class KeyNotFoundException extends TException implements org.apache.thrift.TBase<KeyNotFoundException, KeyNotFoundException._Fields>, java.io.Serializable, Cloneable, Comparable<KeyNotFoundException> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("KeyNotFoundException");
+
+  private static final org.apache.thrift.protocol.TField MSG_FIELD_DESC = new org.apache.thrift.protocol.TField("msg", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new KeyNotFoundExceptionStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new KeyNotFoundExceptionTupleSchemeFactory());
+  }
+
+  private String msg; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    MSG((short)1, "msg");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // MSG
+          return MSG;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.MSG, new org.apache.thrift.meta_data.FieldMetaData("msg", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(KeyNotFoundException.class, metaDataMap);
+  }
+
+  public KeyNotFoundException() {
+  }
+
+  public KeyNotFoundException(
+    String msg)
+  {
+    this();
+    this.msg = msg;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public KeyNotFoundException(KeyNotFoundException other) {
+    if (other.is_set_msg()) {
+      this.msg = other.msg;
+    }
+  }
+
+  public KeyNotFoundException deepCopy() {
+    return new KeyNotFoundException(this);
+  }
+
+  @Override
+  public void clear() {
+    this.msg = null;
+  }
+
+  public String get_msg() {
+    return this.msg;
+  }
+
+  public void set_msg(String msg) {
+    this.msg = msg;
+  }
+
+  public void unset_msg() {
+    this.msg = null;
+  }
+
+  /** Returns true if field msg is set (has been assigned a value) and false otherwise */
+  public boolean is_set_msg() {
+    return this.msg != null;
+  }
+
+  public void set_msg_isSet(boolean value) {
+    if (!value) {
+      this.msg = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case MSG:
+      if (value == null) {
+        unset_msg();
+      } else {
+        set_msg((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case MSG:
+      return get_msg();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case MSG:
+      return is_set_msg();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof KeyNotFoundException)
+      return this.equals((KeyNotFoundException)that);
+    return false;
+  }
+
+  public boolean equals(KeyNotFoundException that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_msg = true && this.is_set_msg();
+    boolean that_present_msg = true && that.is_set_msg();
+    if (this_present_msg || that_present_msg) {
+      if (!(this_present_msg && that_present_msg))
+        return false;
+      if (!this.msg.equals(that.msg))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_msg = true && (is_set_msg());
+    list.add(present_msg);
+    if (present_msg)
+      list.add(msg);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(KeyNotFoundException other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_msg()).compareTo(other.is_set_msg());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_msg()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.msg, other.msg);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("KeyNotFoundException(");
+    boolean first = true;
+
+    sb.append("msg:");
+    if (this.msg == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.msg);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_msg()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class KeyNotFoundExceptionStandardSchemeFactory implements SchemeFactory {
+    public KeyNotFoundExceptionStandardScheme getScheme() {
+      return new KeyNotFoundExceptionStandardScheme();
+    }
+  }
+
+  private static class KeyNotFoundExceptionStandardScheme extends StandardScheme<KeyNotFoundException> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, KeyNotFoundException struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // MSG
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.msg = iprot.readString();
+              struct.set_msg_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, KeyNotFoundException struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.msg != null) {
+        oprot.writeFieldBegin(MSG_FIELD_DESC);
+        oprot.writeString(struct.msg);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class KeyNotFoundExceptionTupleSchemeFactory implements SchemeFactory {
+    public KeyNotFoundExceptionTupleScheme getScheme() {
+      return new KeyNotFoundExceptionTupleScheme();
+    }
+  }
+
+  private static class KeyNotFoundExceptionTupleScheme extends TupleScheme<KeyNotFoundException> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, KeyNotFoundException struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.msg);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, KeyNotFoundException struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.msg = iprot.readString();
+      struct.set_msg_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java b/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java
index 20f0d10..7ff3c74 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java
@@ -365,15 +365,15 @@ public class LSApprovedWorkers implements org.apache.thrift.TBase<LSApprovedWork
           case 1: // APPROVED_WORKERS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map636 = iprot.readMapBegin();
-                struct.approved_workers = new HashMap<String,Integer>(2*_map636.size);
-                String _key637;
-                int _val638;
-                for (int _i639 = 0; _i639 < _map636.size; ++_i639)
+                org.apache.thrift.protocol.TMap _map652 = iprot.readMapBegin();
+                struct.approved_workers = new HashMap<String,Integer>(2*_map652.size);
+                String _key653;
+                int _val654;
+                for (int _i655 = 0; _i655 < _map652.size; ++_i655)
                 {
-                  _key637 = iprot.readString();
-                  _val638 = iprot.readI32();
-                  struct.approved_workers.put(_key637, _val638);
+                  _key653 = iprot.readString();
+                  _val654 = iprot.readI32();
+                  struct.approved_workers.put(_key653, _val654);
                 }
                 iprot.readMapEnd();
               }
@@ -399,10 +399,10 @@ public class LSApprovedWorkers implements org.apache.thrift.TBase<LSApprovedWork
         oprot.writeFieldBegin(APPROVED_WORKERS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.approved_workers.size()));
-          for (Map.Entry<String, Integer> _iter640 : struct.approved_workers.entrySet())
+          for (Map.Entry<String, Integer> _iter656 : struct.approved_workers.entrySet())
           {
-            oprot.writeString(_iter640.getKey());
-            oprot.writeI32(_iter640.getValue());
+            oprot.writeString(_iter656.getKey());
+            oprot.writeI32(_iter656.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -427,10 +427,10 @@ public class LSApprovedWorkers implements org.apache.thrift.TBase<LSApprovedWork
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.approved_workers.size());
-        for (Map.Entry<String, Integer> _iter641 : struct.approved_workers.entrySet())
+        for (Map.Entry<String, Integer> _iter657 : struct.approved_workers.entrySet())
         {
-          oprot.writeString(_iter641.getKey());
-          oprot.writeI32(_iter641.getValue());
+          oprot.writeString(_iter657.getKey());
+          oprot.writeI32(_iter657.getValue());
         }
       }
     }
@@ -439,15 +439,15 @@ public class LSApprovedWorkers implements org.apache.thrift.TBase<LSApprovedWork
     public void read(org.apache.thrift.protocol.TProtocol prot, LSApprovedWorkers struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map642 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32());
-        struct.approved_workers = new HashMap<String,Integer>(2*_map642.size);
-        String _key643;
-        int _val644;
-        for (int _i645 = 0; _i645 < _map642.size; ++_i645)
+        org.apache.thrift.protocol.TMap _map658 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32());
+        struct.approved_workers = new HashMap<String,Integer>(2*_map658.size);
+        String _key659;
+        int _val660;
+        for (int _i661 = 0; _i661 < _map658.size; ++_i661)
         {
-          _key643 = iprot.readString();
-          _val644 = iprot.readI32();
-          struct.approved_workers.put(_key643, _val644);
+          _key659 = iprot.readString();
+          _val660 = iprot.readI32();
+          struct.approved_workers.put(_key659, _val660);
         }
       }
       struct.set_approved_workers_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java b/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java
index de4c803..d655d04 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java
@@ -376,16 +376,16 @@ public class LSSupervisorAssignments implements org.apache.thrift.TBase<LSSuperv
           case 1: // ASSIGNMENTS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map646 = iprot.readMapBegin();
-                struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map646.size);
-                int _key647;
-                LocalAssignment _val648;
-                for (int _i649 = 0; _i649 < _map646.size; ++_i649)
+                org.apache.thrift.protocol.TMap _map662 = iprot.readMapBegin();
+                struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map662.size);
+                int _key663;
+                LocalAssignment _val664;
+                for (int _i665 = 0; _i665 < _map662.size; ++_i665)
                 {
-                  _key647 = iprot.readI32();
-                  _val648 = new LocalAssignment();
-                  _val648.read(iprot);
-                  struct.assignments.put(_key647, _val648);
+                  _key663 = iprot.readI32();
+                  _val664 = new LocalAssignment();
+                  _val664.read(iprot);
+                  struct.assignments.put(_key663, _val664);
                 }
                 iprot.readMapEnd();
               }
@@ -411,10 +411,10 @@ public class LSSupervisorAssignments implements org.apache.thrift.TBase<LSSuperv
         oprot.writeFieldBegin(ASSIGNMENTS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRUCT, struct.assignments.size()));
-          for (Map.Entry<Integer, LocalAssignment> _iter650 : struct.assignments.entrySet())
+          for (Map.Entry<Integer, LocalAssignment> _iter666 : struct.assignments.entrySet())
           {
-            oprot.writeI32(_iter650.getKey());
-            _iter650.getValue().write(oprot);
+            oprot.writeI32(_iter666.getKey());
+            _iter666.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -439,10 +439,10 @@ public class LSSupervisorAssignments implements org.apache.thrift.TBase<LSSuperv
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.assignments.size());
-        for (Map.Entry<Integer, LocalAssignment> _iter651 : struct.assignments.entrySet())
+        for (Map.Entry<Integer, LocalAssignment> _iter667 : struct.assignments.entrySet())
         {
-          oprot.writeI32(_iter651.getKey());
-          _iter651.getValue().write(oprot);
+          oprot.writeI32(_iter667.getKey());
+          _iter667.getValue().write(oprot);
         }
       }
     }
@@ -451,16 +451,16 @@ public class LSSupervisorAssignments implements org.apache.thrift.TBase<LSSuperv
     public void read(org.apache.thrift.protocol.TProtocol prot, LSSupervisorAssignments struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map652 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map652.size);
-        int _key653;
-        LocalAssignment _val654;
-        for (int _i655 = 0; _i655 < _map652.size; ++_i655)
+        org.apache.thrift.protocol.TMap _map668 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map668.size);
+        int _key669;
+        LocalAssignment _val670;
+        for (int _i671 = 0; _i671 < _map668.size; ++_i671)
         {
-          _key653 = iprot.readI32();
-          _val654 = new LocalAssignment();
-          _val654.read(iprot);
-          struct.assignments.put(_key653, _val654);
+          _key669 = iprot.readI32();
+          _val670 = new LocalAssignment();
+          _val670.read(iprot);
+          struct.assignments.put(_key669, _val670);
         }
       }
       struct.set_assignments_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/generated/LSTopoHistory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LSTopoHistory.java b/storm-core/src/jvm/backtype/storm/generated/LSTopoHistory.java
index 79fea1e..ec2931f 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LSTopoHistory.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LSTopoHistory.java
@@ -656,13 +656,13 @@ public class LSTopoHistory implements org.apache.thrift.TBase<LSTopoHistory, LST
           case 3: // USERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list664 = iprot.readListBegin();
-                struct.users = new ArrayList<String>(_list664.size);
-                String _elem665;
-                for (int _i666 = 0; _i666 < _list664.size; ++_i666)
+                org.apache.thrift.protocol.TList _list680 = iprot.readListBegin();
+                struct.users = new ArrayList<String>(_list680.size);
+                String _elem681;
+                for (int _i682 = 0; _i682 < _list680.size; ++_i682)
                 {
-                  _elem665 = iprot.readString();
-                  struct.users.add(_elem665);
+                  _elem681 = iprot.readString();
+                  struct.users.add(_elem681);
                 }
                 iprot.readListEnd();
               }
@@ -674,13 +674,13 @@ public class LSTopoHistory implements org.apache.thrift.TBase<LSTopoHistory, LST
           case 4: // GROUPS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list667 = iprot.readListBegin();
-                struct.groups = new ArrayList<String>(_list667.size);
-                String _elem668;
-                for (int _i669 = 0; _i669 < _list667.size; ++_i669)
+                org.apache.thrift.protocol.TList _list683 = iprot.readListBegin();
+                struct.groups = new ArrayList<String>(_list683.size);
+                String _elem684;
+                for (int _i685 = 0; _i685 < _list683.size; ++_i685)
                 {
-                  _elem668 = iprot.readString();
-                  struct.groups.add(_elem668);
+                  _elem684 = iprot.readString();
+                  struct.groups.add(_elem684);
                 }
                 iprot.readListEnd();
               }
@@ -714,9 +714,9 @@ public class LSTopoHistory implements org.apache.thrift.TBase<LSTopoHistory, LST
         oprot.writeFieldBegin(USERS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.users.size()));
-          for (String _iter670 : struct.users)
+          for (String _iter686 : struct.users)
           {
-            oprot.writeString(_iter670);
+            oprot.writeString(_iter686);
           }
           oprot.writeListEnd();
         }
@@ -726,9 +726,9 @@ public class LSTopoHistory implements org.apache.thrift.TBase<LSTopoHistory, LST
         oprot.writeFieldBegin(GROUPS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.groups.size()));
-          for (String _iter671 : struct.groups)
+          for (String _iter687 : struct.groups)
           {
-            oprot.writeString(_iter671);
+            oprot.writeString(_iter687);
           }
           oprot.writeListEnd();
         }
@@ -755,16 +755,16 @@ public class LSTopoHistory implements org.apache.thrift.TBase<LSTopoHistory, LST
       oprot.writeI64(struct.time_stamp);
       {
         oprot.writeI32(struct.users.size());
-        for (String _iter672 : struct.users)
+        for (String _iter688 : struct.users)
         {
-          oprot.writeString(_iter672);
+          oprot.writeString(_iter688);
         }
       }
       {
         oprot.writeI32(struct.groups.size());
-        for (String _iter673 : struct.groups)
+        for (String _iter689 : struct.groups)
         {
-          oprot.writeString(_iter673);
+          oprot.writeString(_iter689);
         }
       }
     }
@@ -777,24 +777,24 @@ public class LSTopoHistory implements org.apache.thrift.TBase<LSTopoHistory, LST
       struct.time_stamp = iprot.readI64();
       struct.set_time_stamp_isSet(true);
       {
-        org.apache.thrift.protocol.TList _list674 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.users = new ArrayList<String>(_list674.size);
-        String _elem675;
-        for (int _i676 = 0; _i676 < _list674.size; ++_i676)
+        org.apache.thrift.protocol.TList _list690 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.users = new ArrayList<String>(_list690.size);
+        String _elem691;
+        for (int _i692 = 0; _i692 < _list690.size; ++_i692)
         {
-          _elem675 = iprot.readString();
-          struct.users.add(_elem675);
+          _elem691 = iprot.readString();
+          struct.users.add(_elem691);
         }
       }
       struct.set_users_isSet(true);
       {
-        org.apache.thrift.protocol.TList _list677 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.groups = new ArrayList<String>(_list677.size);
-        String _elem678;
-        for (int _i679 = 0; _i679 < _list677.size; ++_i679)
+        org.apache.thrift.protocol.TList _list693 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.groups = new ArrayList<String>(_list693.size);
+        String _elem694;
+        for (int _i695 = 0; _i695 < _list693.size; ++_i695)
         {
-          _elem678 = iprot.readString();
-          struct.groups.add(_elem678);
+          _elem694 = iprot.readString();
+          struct.groups.add(_elem694);
         }
       }
       struct.set_groups_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/generated/LSTopoHistoryList.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LSTopoHistoryList.java b/storm-core/src/jvm/backtype/storm/generated/LSTopoHistoryList.java
index 962ece6..a386cf5 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LSTopoHistoryList.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LSTopoHistoryList.java
@@ -371,14 +371,14 @@ public class LSTopoHistoryList implements org.apache.thrift.TBase<LSTopoHistoryL
           case 1: // TOPO_HISTORY
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list680 = iprot.readListBegin();
-                struct.topo_history = new ArrayList<LSTopoHistory>(_list680.size);
-                LSTopoHistory _elem681;
-                for (int _i682 = 0; _i682 < _list680.size; ++_i682)
+                org.apache.thrift.protocol.TList _list696 = iprot.readListBegin();
+                struct.topo_history = new ArrayList<LSTopoHistory>(_list696.size);
+                LSTopoHistory _elem697;
+                for (int _i698 = 0; _i698 < _list696.size; ++_i698)
                 {
-                  _elem681 = new LSTopoHistory();
-                  _elem681.read(iprot);
-                  struct.topo_history.add(_elem681);
+                  _elem697 = new LSTopoHistory();
+                  _elem697.read(iprot);
+                  struct.topo_history.add(_elem697);
                 }
                 iprot.readListEnd();
               }
@@ -404,9 +404,9 @@ public class LSTopoHistoryList implements org.apache.thrift.TBase<LSTopoHistoryL
         oprot.writeFieldBegin(TOPO_HISTORY_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.topo_history.size()));
-          for (LSTopoHistory _iter683 : struct.topo_history)
+          for (LSTopoHistory _iter699 : struct.topo_history)
           {
-            _iter683.write(oprot);
+            _iter699.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -431,9 +431,9 @@ public class LSTopoHistoryList implements org.apache.thrift.TBase<LSTopoHistoryL
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.topo_history.size());
-        for (LSTopoHistory _iter684 : struct.topo_history)
+        for (LSTopoHistory _iter700 : struct.topo_history)
         {
-          _iter684.write(oprot);
+          _iter700.write(oprot);
         }
       }
     }
@@ -442,14 +442,14 @@ public class LSTopoHistoryList implements org.apache.thrift.TBase<LSTopoHistoryL
     public void read(org.apache.thrift.protocol.TProtocol prot, LSTopoHistoryList struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list685 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.topo_history = new ArrayList<LSTopoHistory>(_list685.size);
-        LSTopoHistory _elem686;
-        for (int _i687 = 0; _i687 < _list685.size; ++_i687)
+        org.apache.thrift.protocol.TList _list701 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.topo_history = new ArrayList<LSTopoHistory>(_list701.size);
+        LSTopoHistory _elem702;
+        for (int _i703 = 0; _i703 < _list701.size; ++_i703)
         {
-          _elem686 = new LSTopoHistory();
-          _elem686.read(iprot);
-          struct.topo_history.add(_elem686);
+          _elem702 = new LSTopoHistory();
+          _elem702.read(iprot);
+          struct.topo_history.add(_elem702);
         }
       }
       struct.set_topo_history_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java b/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java
index d6e7c36..9252270 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java
@@ -638,14 +638,14 @@ public class LSWorkerHeartbeat implements org.apache.thrift.TBase<LSWorkerHeartb
           case 3: // EXECUTORS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list656 = iprot.readListBegin();
-                struct.executors = new ArrayList<ExecutorInfo>(_list656.size);
-                ExecutorInfo _elem657;
-                for (int _i658 = 0; _i658 < _list656.size; ++_i658)
+                org.apache.thrift.protocol.TList _list672 = iprot.readListBegin();
+                struct.executors = new ArrayList<ExecutorInfo>(_list672.size);
+                ExecutorInfo _elem673;
+                for (int _i674 = 0; _i674 < _list672.size; ++_i674)
                 {
-                  _elem657 = new ExecutorInfo();
-                  _elem657.read(iprot);
-                  struct.executors.add(_elem657);
+                  _elem673 = new ExecutorInfo();
+                  _elem673.read(iprot);
+                  struct.executors.add(_elem673);
                 }
                 iprot.readListEnd();
               }
@@ -687,9 +687,9 @@ public class LSWorkerHeartbeat implements org.apache.thrift.TBase<LSWorkerHeartb
         oprot.writeFieldBegin(EXECUTORS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size()));
-          for (ExecutorInfo _iter659 : struct.executors)
+          for (ExecutorInfo _iter675 : struct.executors)
           {
-            _iter659.write(oprot);
+            _iter675.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -719,9 +719,9 @@ public class LSWorkerHeartbeat implements org.apache.thrift.TBase<LSWorkerHeartb
       oprot.writeString(struct.topology_id);
       {
         oprot.writeI32(struct.executors.size());
-        for (ExecutorInfo _iter660 : struct.executors)
+        for (ExecutorInfo _iter676 : struct.executors)
         {
-          _iter660.write(oprot);
+          _iter676.write(oprot);
         }
       }
       oprot.writeI32(struct.port);
@@ -735,14 +735,14 @@ public class LSWorkerHeartbeat implements org.apache.thrift.TBase<LSWorkerHeartb
       struct.topology_id = iprot.readString();
       struct.set_topology_id_isSet(true);
       {
-        org.apache.thrift.protocol.TList _list661 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.executors = new ArrayList<ExecutorInfo>(_list661.size);
-        ExecutorInfo _elem662;
-        for (int _i663 = 0; _i663 < _list661.size; ++_i663)
+        org.apache.thrift.protocol.TList _list677 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.executors = new ArrayList<ExecutorInfo>(_list677.size);
+        ExecutorInfo _elem678;
+        for (int _i679 = 0; _i679 < _list677.size; ++_i679)
         {
-          _elem662 = new ExecutorInfo();
-          _elem662.read(iprot);
-          struct.executors.add(_elem662);
+          _elem678 = new ExecutorInfo();
+          _elem678.read(iprot);
+          struct.executors.add(_elem678);
         }
       }
       struct.set_executors_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/generated/ListBlobsResult.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ListBlobsResult.java b/storm-core/src/jvm/backtype/storm/generated/ListBlobsResult.java
new file mode 100644
index 0000000..94b27f8
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/ListBlobsResult.java
@@ -0,0 +1,556 @@
+/**
+ * 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.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class ListBlobsResult implements org.apache.thrift.TBase<ListBlobsResult, ListBlobsResult._Fields>, java.io.Serializable, Cloneable, Comparable<ListBlobsResult> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ListBlobsResult");
+
+  private static final org.apache.thrift.protocol.TField KEYS_FIELD_DESC = new org.apache.thrift.protocol.TField("keys", org.apache.thrift.protocol.TType.LIST, (short)1);
+  private static final org.apache.thrift.protocol.TField SESSION_FIELD_DESC = new org.apache.thrift.protocol.TField("session", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ListBlobsResultStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ListBlobsResultTupleSchemeFactory());
+  }
+
+  private List<String> keys; // required
+  private String session; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    KEYS((short)1, "keys"),
+    SESSION((short)2, "session");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // KEYS
+          return KEYS;
+        case 2: // SESSION
+          return SESSION;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.KEYS, new org.apache.thrift.meta_data.FieldMetaData("keys", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.SESSION, new org.apache.thrift.meta_data.FieldMetaData("session", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ListBlobsResult.class, metaDataMap);
+  }
+
+  public ListBlobsResult() {
+  }
+
+  public ListBlobsResult(
+    List<String> keys,
+    String session)
+  {
+    this();
+    this.keys = keys;
+    this.session = session;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ListBlobsResult(ListBlobsResult other) {
+    if (other.is_set_keys()) {
+      List<String> __this__keys = new ArrayList<String>(other.keys);
+      this.keys = __this__keys;
+    }
+    if (other.is_set_session()) {
+      this.session = other.session;
+    }
+  }
+
+  public ListBlobsResult deepCopy() {
+    return new ListBlobsResult(this);
+  }
+
+  @Override
+  public void clear() {
+    this.keys = null;
+    this.session = null;
+  }
+
+  public int get_keys_size() {
+    return (this.keys == null) ? 0 : this.keys.size();
+  }
+
+  public java.util.Iterator<String> get_keys_iterator() {
+    return (this.keys == null) ? null : this.keys.iterator();
+  }
+
+  public void add_to_keys(String elem) {
+    if (this.keys == null) {
+      this.keys = new ArrayList<String>();
+    }
+    this.keys.add(elem);
+  }
+
+  public List<String> get_keys() {
+    return this.keys;
+  }
+
+  public void set_keys(List<String> keys) {
+    this.keys = keys;
+  }
+
+  public void unset_keys() {
+    this.keys = null;
+  }
+
+  /** Returns true if field keys is set (has been assigned a value) and false otherwise */
+  public boolean is_set_keys() {
+    return this.keys != null;
+  }
+
+  public void set_keys_isSet(boolean value) {
+    if (!value) {
+      this.keys = null;
+    }
+  }
+
+  public String get_session() {
+    return this.session;
+  }
+
+  public void set_session(String session) {
+    this.session = session;
+  }
+
+  public void unset_session() {
+    this.session = null;
+  }
+
+  /** Returns true if field session is set (has been assigned a value) and false otherwise */
+  public boolean is_set_session() {
+    return this.session != null;
+  }
+
+  public void set_session_isSet(boolean value) {
+    if (!value) {
+      this.session = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case KEYS:
+      if (value == null) {
+        unset_keys();
+      } else {
+        set_keys((List<String>)value);
+      }
+      break;
+
+    case SESSION:
+      if (value == null) {
+        unset_session();
+      } else {
+        set_session((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case KEYS:
+      return get_keys();
+
+    case SESSION:
+      return get_session();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case KEYS:
+      return is_set_keys();
+    case SESSION:
+      return is_set_session();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ListBlobsResult)
+      return this.equals((ListBlobsResult)that);
+    return false;
+  }
+
+  public boolean equals(ListBlobsResult that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_keys = true && this.is_set_keys();
+    boolean that_present_keys = true && that.is_set_keys();
+    if (this_present_keys || that_present_keys) {
+      if (!(this_present_keys && that_present_keys))
+        return false;
+      if (!this.keys.equals(that.keys))
+        return false;
+    }
+
+    boolean this_present_session = true && this.is_set_session();
+    boolean that_present_session = true && that.is_set_session();
+    if (this_present_session || that_present_session) {
+      if (!(this_present_session && that_present_session))
+        return false;
+      if (!this.session.equals(that.session))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_keys = true && (is_set_keys());
+    list.add(present_keys);
+    if (present_keys)
+      list.add(keys);
+
+    boolean present_session = true && (is_set_session());
+    list.add(present_session);
+    if (present_session)
+      list.add(session);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ListBlobsResult other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_keys()).compareTo(other.is_set_keys());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_keys()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.keys, other.keys);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_session()).compareTo(other.is_set_session());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_session()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.session, other.session);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ListBlobsResult(");
+    boolean first = true;
+
+    sb.append("keys:");
+    if (this.keys == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.keys);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("session:");
+    if (this.session == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.session);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_keys()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'keys' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_session()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'session' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ListBlobsResultStandardSchemeFactory implements SchemeFactory {
+    public ListBlobsResultStandardScheme getScheme() {
+      return new ListBlobsResultStandardScheme();
+    }
+  }
+
+  private static class ListBlobsResultStandardScheme extends StandardScheme<ListBlobsResult> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ListBlobsResult struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // KEYS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list496 = iprot.readListBegin();
+                struct.keys = new ArrayList<String>(_list496.size);
+                String _elem497;
+                for (int _i498 = 0; _i498 < _list496.size; ++_i498)
+                {
+                  _elem497 = iprot.readString();
+                  struct.keys.add(_elem497);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_keys_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // SESSION
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.session = iprot.readString();
+              struct.set_session_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ListBlobsResult struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.keys != null) {
+        oprot.writeFieldBegin(KEYS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.keys.size()));
+          for (String _iter499 : struct.keys)
+          {
+            oprot.writeString(_iter499);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.session != null) {
+        oprot.writeFieldBegin(SESSION_FIELD_DESC);
+        oprot.writeString(struct.session);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ListBlobsResultTupleSchemeFactory implements SchemeFactory {
+    public ListBlobsResultTupleScheme getScheme() {
+      return new ListBlobsResultTupleScheme();
+    }
+  }
+
+  private static class ListBlobsResultTupleScheme extends TupleScheme<ListBlobsResult> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ListBlobsResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.keys.size());
+        for (String _iter500 : struct.keys)
+        {
+          oprot.writeString(_iter500);
+        }
+      }
+      oprot.writeString(struct.session);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ListBlobsResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TList _list501 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.keys = new ArrayList<String>(_list501.size);
+        String _elem502;
+        for (int _i503 = 0; _i503 < _list501.size; ++_i503)
+        {
+          _elem502 = iprot.readString();
+          struct.keys.add(_elem502);
+        }
+      }
+      struct.set_keys_isSet(true);
+      struct.session = iprot.readString();
+      struct.set_session_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java b/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java
index a36e654..68bdd1d 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java
@@ -549,14 +549,14 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
           case 2: // EXECUTORS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list628 = iprot.readListBegin();
-                struct.executors = new ArrayList<ExecutorInfo>(_list628.size);
-                ExecutorInfo _elem629;
-                for (int _i630 = 0; _i630 < _list628.size; ++_i630)
+                org.apache.thrift.protocol.TList _list644 = iprot.readListBegin();
+                struct.executors = new ArrayList<ExecutorInfo>(_list644.size);
+                ExecutorInfo _elem645;
+                for (int _i646 = 0; _i646 < _list644.size; ++_i646)
                 {
-                  _elem629 = new ExecutorInfo();
-                  _elem629.read(iprot);
-                  struct.executors.add(_elem629);
+                  _elem645 = new ExecutorInfo();
+                  _elem645.read(iprot);
+                  struct.executors.add(_elem645);
                 }
                 iprot.readListEnd();
               }
@@ -596,9 +596,9 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
         oprot.writeFieldBegin(EXECUTORS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size()));
-          for (ExecutorInfo _iter631 : struct.executors)
+          for (ExecutorInfo _iter647 : struct.executors)
           {
-            _iter631.write(oprot);
+            _iter647.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -631,9 +631,9 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
       oprot.writeString(struct.topology_id);
       {
         oprot.writeI32(struct.executors.size());
-        for (ExecutorInfo _iter632 : struct.executors)
+        for (ExecutorInfo _iter648 : struct.executors)
         {
-          _iter632.write(oprot);
+          _iter648.write(oprot);
         }
       }
       BitSet optionals = new BitSet();
@@ -652,14 +652,14 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
       struct.topology_id = iprot.readString();
       struct.set_topology_id_isSet(true);
       {
-        org.apache.thrift.protocol.TList _list633 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.executors = new ArrayList<ExecutorInfo>(_list633.size);
-        ExecutorInfo _elem634;
-        for (int _i635 = 0; _i635 < _list633.size; ++_i635)
+        org.apache.thrift.protocol.TList _list649 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.executors = new ArrayList<ExecutorInfo>(_list649.size);
+        ExecutorInfo _elem650;
+        for (int _i651 = 0; _i651 < _list649.size; ++_i651)
         {
-          _elem634 = new ExecutorInfo();
-          _elem634.read(iprot);
-          struct.executors.add(_elem634);
+          _elem650 = new ExecutorInfo();
+          _elem650.read(iprot);
+          struct.executors.add(_elem650);
         }
       }
       struct.set_executors_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java b/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java
index 7340926..538018c 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java
@@ -376,16 +376,16 @@ public class LocalStateData implements org.apache.thrift.TBase<LocalStateData, L
           case 1: // SERIALIZED_PARTS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map618 = iprot.readMapBegin();
-                struct.serialized_parts = new HashMap<String,ThriftSerializedObject>(2*_map618.size);
-                String _key619;
-                ThriftSerializedObject _val620;
-                for (int _i621 = 0; _i621 < _map618.size; ++_i621)
+                org.apache.thrift.protocol.TMap _map634 = iprot.readMapBegin();
+                struct.serialized_parts = new HashMap<String,ThriftSerializedObject>(2*_map634.size);
+                String _key635;
+                ThriftSerializedObject _val636;
+                for (int _i637 = 0; _i637 < _map634.size; ++_i637)
                 {
-                  _key619 = iprot.readString();
-                  _val620 = new ThriftSerializedObject();
-                  _val620.read(iprot);
-                  struct.serialized_parts.put(_key619, _val620);
+                  _key635 = iprot.readString();
+                  _val636 = new ThriftSerializedObject();
+                  _val636.read(iprot);
+                  struct.serialized_parts.put(_key635, _val636);
                 }
                 iprot.readMapEnd();
               }
@@ -411,10 +411,10 @@ public class LocalStateData implements org.apache.thrift.TBase<LocalStateData, L
         oprot.writeFieldBegin(SERIALIZED_PARTS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.serialized_parts.size()));
-          for (Map.Entry<String, ThriftSerializedObject> _iter622 : struct.serialized_parts.entrySet())
+          for (Map.Entry<String, ThriftSerializedObject> _iter638 : struct.serialized_parts.entrySet())
           {
-            oprot.writeString(_iter622.getKey());
-            _iter622.getValue().write(oprot);
+            oprot.writeString(_iter638.getKey());
+            _iter638.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -439,10 +439,10 @@ public class LocalStateData implements org.apache.thrift.TBase<LocalStateData, L
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.serialized_parts.size());
-        for (Map.Entry<String, ThriftSerializedObject> _iter623 : struct.serialized_parts.entrySet())
+        for (Map.Entry<String, ThriftSerializedObject> _iter639 : struct.serialized_parts.entrySet())
         {
-          oprot.writeString(_iter623.getKey());
-          _iter623.getValue().write(oprot);
+          oprot.writeString(_iter639.getKey());
+          _iter639.getValue().write(oprot);
         }
       }
     }
@@ -451,16 +451,16 @@ public class LocalStateData implements org.apache.thrift.TBase<LocalStateData, L
     public void read(org.apache.thrift.protocol.TProtocol prot, LocalStateData struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map624 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.serialized_parts = new HashMap<String,ThriftSerializedObject>(2*_map624.size);
-        String _key625;
-        ThriftSerializedObject _val626;
-        for (int _i627 = 0; _i627 < _map624.size; ++_i627)
+        org.apache.thrift.protocol.TMap _map640 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.serialized_parts = new HashMap<String,ThriftSerializedObject>(2*_map640.size);
+        String _key641;
+        ThriftSerializedObject _val642;
+        for (int _i643 = 0; _i643 < _map640.size; ++_i643)
         {
-          _key625 = iprot.readString();
-          _val626 = new ThriftSerializedObject();
-          _val626.read(iprot);
-          struct.serialized_parts.put(_key625, _val626);
+          _key641 = iprot.readString();
+          _val642 = new ThriftSerializedObject();
+          _val642.read(iprot);
+          struct.serialized_parts.put(_key641, _val642);
         }
       }
       struct.set_serialized_parts_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/generated/LogConfig.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LogConfig.java b/storm-core/src/jvm/backtype/storm/generated/LogConfig.java
index 53bc326..36090f7 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LogConfig.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LogConfig.java
@@ -368,16 +368,16 @@ public class LogConfig implements org.apache.thrift.TBase<LogConfig, LogConfig._
           case 2: // NAMED_LOGGER_LEVEL
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map688 = iprot.readMapBegin();
-                struct.named_logger_level = new HashMap<String,LogLevel>(2*_map688.size);
-                String _key689;
-                LogLevel _val690;
-                for (int _i691 = 0; _i691 < _map688.size; ++_i691)
+                org.apache.thrift.protocol.TMap _map704 = iprot.readMapBegin();
+                struct.named_logger_level = new HashMap<String,LogLevel>(2*_map704.size);
+                String _key705;
+                LogLevel _val706;
+                for (int _i707 = 0; _i707 < _map704.size; ++_i707)
                 {
-                  _key689 = iprot.readString();
-                  _val690 = new LogLevel();
-                  _val690.read(iprot);
-                  struct.named_logger_level.put(_key689, _val690);
+                  _key705 = iprot.readString();
+                  _val706 = new LogLevel();
+                  _val706.read(iprot);
+                  struct.named_logger_level.put(_key705, _val706);
                 }
                 iprot.readMapEnd();
               }
@@ -404,10 +404,10 @@ public class LogConfig implements org.apache.thrift.TBase<LogConfig, LogConfig._
           oprot.writeFieldBegin(NAMED_LOGGER_LEVEL_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.named_logger_level.size()));
-            for (Map.Entry<String, LogLevel> _iter692 : struct.named_logger_level.entrySet())
+            for (Map.Entry<String, LogLevel> _iter708 : struct.named_logger_level.entrySet())
             {
-              oprot.writeString(_iter692.getKey());
-              _iter692.getValue().write(oprot);
+              oprot.writeString(_iter708.getKey());
+              _iter708.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -439,10 +439,10 @@ public class LogConfig implements org.apache.thrift.TBase<LogConfig, LogConfig._
       if (struct.is_set_named_logger_level()) {
         {
           oprot.writeI32(struct.named_logger_level.size());
-          for (Map.Entry<String, LogLevel> _iter693 : struct.named_logger_level.entrySet())
+          for (Map.Entry<String, LogLevel> _iter709 : struct.named_logger_level.entrySet())
           {
-            oprot.writeString(_iter693.getKey());
-            _iter693.getValue().write(oprot);
+            oprot.writeString(_iter709.getKey());
+            _iter709.getValue().write(oprot);
           }
         }
       }
@@ -454,16 +454,16 @@ public class LogConfig implements org.apache.thrift.TBase<LogConfig, LogConfig._
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TMap _map694 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.named_logger_level = new HashMap<String,LogLevel>(2*_map694.size);
-          String _key695;
-          LogLevel _val696;
-          for (int _i697 = 0; _i697 < _map694.size; ++_i697)
+          org.apache.thrift.protocol.TMap _map710 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.named_logger_level = new HashMap<String,LogLevel>(2*_map710.size);
+          String _key711;
+          LogLevel _val712;
+          for (int _i713 = 0; _i713 < _map710.size; ++_i713)
           {
-            _key695 = iprot.readString();
-            _val696 = new LogLevel();
-            _val696.read(iprot);
-            struct.named_logger_level.put(_key695, _val696);
+            _key711 = iprot.readString();
+            _val712 = new LogLevel();
+            _val712.read(iprot);
+            struct.named_logger_level.put(_key711, _val712);
           }
         }
         struct.set_named_logger_level_isSet(true);


[06/17] storm git commit: Blobstore API STORM- 876

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/Nimbus.java b/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
index 98d2d1c..042ef0e 100644
--- a/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
+++ b/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
@@ -94,6 +94,34 @@ public class Nimbus {
 
     public void uploadNewCredentials(String name, Credentials creds) throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException;
 
+    public String beginCreateBlob(String key, SettableBlobMeta meta) throws AuthorizationException, KeyAlreadyExistsException, org.apache.thrift.TException;
+
+    public String beginUpdateBlob(String key) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException;
+
+    public void uploadBlobChunk(String session, ByteBuffer chunk) throws AuthorizationException, org.apache.thrift.TException;
+
+    public void finishBlobUpload(String session) throws AuthorizationException, org.apache.thrift.TException;
+
+    public void cancelBlobUpload(String session) throws AuthorizationException, org.apache.thrift.TException;
+
+    public ReadableBlobMeta getBlobMeta(String key) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException;
+
+    public void setBlobMeta(String key, SettableBlobMeta meta) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException;
+
+    public BeginDownloadResult beginBlobDownload(String key) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException;
+
+    public ByteBuffer downloadBlobChunk(String session) throws AuthorizationException, org.apache.thrift.TException;
+
+    public void deleteBlob(String key) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException;
+
+    public ListBlobsResult listBlobs(String session) throws org.apache.thrift.TException;
+
+    public int getBlobReplication(String key) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException;
+
+    public int updateBlobReplication(String key, int replication) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException;
+
+    public void createStateInZookeeper(String key) throws org.apache.thrift.TException;
+
     public String beginFileUpload() throws AuthorizationException, org.apache.thrift.TException;
 
     public void uploadChunk(String location, ByteBuffer chunk) throws AuthorizationException, org.apache.thrift.TException;
@@ -164,6 +192,34 @@ public class Nimbus {
 
     public void uploadNewCredentials(String name, Credentials creds, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void beginCreateBlob(String key, SettableBlobMeta meta, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void beginUpdateBlob(String key, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void uploadBlobChunk(String session, ByteBuffer chunk, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void finishBlobUpload(String session, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void cancelBlobUpload(String session, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getBlobMeta(String key, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void setBlobMeta(String key, SettableBlobMeta meta, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void beginBlobDownload(String key, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void downloadBlobChunk(String session, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void deleteBlob(String key, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void listBlobs(String session, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getBlobReplication(String key, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void updateBlobReplication(String key, int replication, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void createStateInZookeeper(String key, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
     public void beginFileUpload(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void uploadChunk(String location, ByteBuffer chunk, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
@@ -565,6 +621,374 @@ public class Nimbus {
       return;
     }
 
+    public String beginCreateBlob(String key, SettableBlobMeta meta) throws AuthorizationException, KeyAlreadyExistsException, org.apache.thrift.TException
+    {
+      send_beginCreateBlob(key, meta);
+      return recv_beginCreateBlob();
+    }
+
+    public void send_beginCreateBlob(String key, SettableBlobMeta meta) throws org.apache.thrift.TException
+    {
+      beginCreateBlob_args args = new beginCreateBlob_args();
+      args.set_key(key);
+      args.set_meta(meta);
+      sendBase("beginCreateBlob", args);
+    }
+
+    public String recv_beginCreateBlob() throws AuthorizationException, KeyAlreadyExistsException, org.apache.thrift.TException
+    {
+      beginCreateBlob_result result = new beginCreateBlob_result();
+      receiveBase(result, "beginCreateBlob");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      if (result.kae != null) {
+        throw result.kae;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "beginCreateBlob failed: unknown result");
+    }
+
+    public String beginUpdateBlob(String key) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      send_beginUpdateBlob(key);
+      return recv_beginUpdateBlob();
+    }
+
+    public void send_beginUpdateBlob(String key) throws org.apache.thrift.TException
+    {
+      beginUpdateBlob_args args = new beginUpdateBlob_args();
+      args.set_key(key);
+      sendBase("beginUpdateBlob", args);
+    }
+
+    public String recv_beginUpdateBlob() throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      beginUpdateBlob_result result = new beginUpdateBlob_result();
+      receiveBase(result, "beginUpdateBlob");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      if (result.knf != null) {
+        throw result.knf;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "beginUpdateBlob failed: unknown result");
+    }
+
+    public void uploadBlobChunk(String session, ByteBuffer chunk) throws AuthorizationException, org.apache.thrift.TException
+    {
+      send_uploadBlobChunk(session, chunk);
+      recv_uploadBlobChunk();
+    }
+
+    public void send_uploadBlobChunk(String session, ByteBuffer chunk) throws org.apache.thrift.TException
+    {
+      uploadBlobChunk_args args = new uploadBlobChunk_args();
+      args.set_session(session);
+      args.set_chunk(chunk);
+      sendBase("uploadBlobChunk", args);
+    }
+
+    public void recv_uploadBlobChunk() throws AuthorizationException, org.apache.thrift.TException
+    {
+      uploadBlobChunk_result result = new uploadBlobChunk_result();
+      receiveBase(result, "uploadBlobChunk");
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      return;
+    }
+
+    public void finishBlobUpload(String session) throws AuthorizationException, org.apache.thrift.TException
+    {
+      send_finishBlobUpload(session);
+      recv_finishBlobUpload();
+    }
+
+    public void send_finishBlobUpload(String session) throws org.apache.thrift.TException
+    {
+      finishBlobUpload_args args = new finishBlobUpload_args();
+      args.set_session(session);
+      sendBase("finishBlobUpload", args);
+    }
+
+    public void recv_finishBlobUpload() throws AuthorizationException, org.apache.thrift.TException
+    {
+      finishBlobUpload_result result = new finishBlobUpload_result();
+      receiveBase(result, "finishBlobUpload");
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      return;
+    }
+
+    public void cancelBlobUpload(String session) throws AuthorizationException, org.apache.thrift.TException
+    {
+      send_cancelBlobUpload(session);
+      recv_cancelBlobUpload();
+    }
+
+    public void send_cancelBlobUpload(String session) throws org.apache.thrift.TException
+    {
+      cancelBlobUpload_args args = new cancelBlobUpload_args();
+      args.set_session(session);
+      sendBase("cancelBlobUpload", args);
+    }
+
+    public void recv_cancelBlobUpload() throws AuthorizationException, org.apache.thrift.TException
+    {
+      cancelBlobUpload_result result = new cancelBlobUpload_result();
+      receiveBase(result, "cancelBlobUpload");
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      return;
+    }
+
+    public ReadableBlobMeta getBlobMeta(String key) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      send_getBlobMeta(key);
+      return recv_getBlobMeta();
+    }
+
+    public void send_getBlobMeta(String key) throws org.apache.thrift.TException
+    {
+      getBlobMeta_args args = new getBlobMeta_args();
+      args.set_key(key);
+      sendBase("getBlobMeta", args);
+    }
+
+    public ReadableBlobMeta recv_getBlobMeta() throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      getBlobMeta_result result = new getBlobMeta_result();
+      receiveBase(result, "getBlobMeta");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      if (result.knf != null) {
+        throw result.knf;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getBlobMeta failed: unknown result");
+    }
+
+    public void setBlobMeta(String key, SettableBlobMeta meta) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      send_setBlobMeta(key, meta);
+      recv_setBlobMeta();
+    }
+
+    public void send_setBlobMeta(String key, SettableBlobMeta meta) throws org.apache.thrift.TException
+    {
+      setBlobMeta_args args = new setBlobMeta_args();
+      args.set_key(key);
+      args.set_meta(meta);
+      sendBase("setBlobMeta", args);
+    }
+
+    public void recv_setBlobMeta() throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      setBlobMeta_result result = new setBlobMeta_result();
+      receiveBase(result, "setBlobMeta");
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      if (result.knf != null) {
+        throw result.knf;
+      }
+      return;
+    }
+
+    public BeginDownloadResult beginBlobDownload(String key) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      send_beginBlobDownload(key);
+      return recv_beginBlobDownload();
+    }
+
+    public void send_beginBlobDownload(String key) throws org.apache.thrift.TException
+    {
+      beginBlobDownload_args args = new beginBlobDownload_args();
+      args.set_key(key);
+      sendBase("beginBlobDownload", args);
+    }
+
+    public BeginDownloadResult recv_beginBlobDownload() throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      beginBlobDownload_result result = new beginBlobDownload_result();
+      receiveBase(result, "beginBlobDownload");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      if (result.knf != null) {
+        throw result.knf;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "beginBlobDownload failed: unknown result");
+    }
+
+    public ByteBuffer downloadBlobChunk(String session) throws AuthorizationException, org.apache.thrift.TException
+    {
+      send_downloadBlobChunk(session);
+      return recv_downloadBlobChunk();
+    }
+
+    public void send_downloadBlobChunk(String session) throws org.apache.thrift.TException
+    {
+      downloadBlobChunk_args args = new downloadBlobChunk_args();
+      args.set_session(session);
+      sendBase("downloadBlobChunk", args);
+    }
+
+    public ByteBuffer recv_downloadBlobChunk() throws AuthorizationException, org.apache.thrift.TException
+    {
+      downloadBlobChunk_result result = new downloadBlobChunk_result();
+      receiveBase(result, "downloadBlobChunk");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "downloadBlobChunk failed: unknown result");
+    }
+
+    public void deleteBlob(String key) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      send_deleteBlob(key);
+      recv_deleteBlob();
+    }
+
+    public void send_deleteBlob(String key) throws org.apache.thrift.TException
+    {
+      deleteBlob_args args = new deleteBlob_args();
+      args.set_key(key);
+      sendBase("deleteBlob", args);
+    }
+
+    public void recv_deleteBlob() throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      deleteBlob_result result = new deleteBlob_result();
+      receiveBase(result, "deleteBlob");
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      if (result.knf != null) {
+        throw result.knf;
+      }
+      return;
+    }
+
+    public ListBlobsResult listBlobs(String session) throws org.apache.thrift.TException
+    {
+      send_listBlobs(session);
+      return recv_listBlobs();
+    }
+
+    public void send_listBlobs(String session) throws org.apache.thrift.TException
+    {
+      listBlobs_args args = new listBlobs_args();
+      args.set_session(session);
+      sendBase("listBlobs", args);
+    }
+
+    public ListBlobsResult recv_listBlobs() throws org.apache.thrift.TException
+    {
+      listBlobs_result result = new listBlobs_result();
+      receiveBase(result, "listBlobs");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "listBlobs failed: unknown result");
+    }
+
+    public int getBlobReplication(String key) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      send_getBlobReplication(key);
+      return recv_getBlobReplication();
+    }
+
+    public void send_getBlobReplication(String key) throws org.apache.thrift.TException
+    {
+      getBlobReplication_args args = new getBlobReplication_args();
+      args.set_key(key);
+      sendBase("getBlobReplication", args);
+    }
+
+    public int recv_getBlobReplication() throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      getBlobReplication_result result = new getBlobReplication_result();
+      receiveBase(result, "getBlobReplication");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      if (result.knf != null) {
+        throw result.knf;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getBlobReplication failed: unknown result");
+    }
+
+    public int updateBlobReplication(String key, int replication) throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      send_updateBlobReplication(key, replication);
+      return recv_updateBlobReplication();
+    }
+
+    public void send_updateBlobReplication(String key, int replication) throws org.apache.thrift.TException
+    {
+      updateBlobReplication_args args = new updateBlobReplication_args();
+      args.set_key(key);
+      args.set_replication(replication);
+      sendBase("updateBlobReplication", args);
+    }
+
+    public int recv_updateBlobReplication() throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException
+    {
+      updateBlobReplication_result result = new updateBlobReplication_result();
+      receiveBase(result, "updateBlobReplication");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      if (result.knf != null) {
+        throw result.knf;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "updateBlobReplication failed: unknown result");
+    }
+
+    public void createStateInZookeeper(String key) throws org.apache.thrift.TException
+    {
+      send_createStateInZookeeper(key);
+      recv_createStateInZookeeper();
+    }
+
+    public void send_createStateInZookeeper(String key) throws org.apache.thrift.TException
+    {
+      createStateInZookeeper_args args = new createStateInZookeeper_args();
+      args.set_key(key);
+      sendBase("createStateInZookeeper", args);
+    }
+
+    public void recv_createStateInZookeeper() throws org.apache.thrift.TException
+    {
+      createStateInZookeeper_result result = new createStateInZookeeper_result();
+      receiveBase(result, "createStateInZookeeper");
+      return;
+    }
+
     public String beginFileUpload() throws AuthorizationException, org.apache.thrift.TException
     {
       send_beginFileUpload();
@@ -1459,88 +1883,94 @@ public class Nimbus {
       }
     }
 
-    public void beginFileUpload(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+    public void beginCreateBlob(String key, SettableBlobMeta meta, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      beginFileUpload_call method_call = new beginFileUpload_call(resultHandler, this, ___protocolFactory, ___transport);
+      beginCreateBlob_call method_call = new beginCreateBlob_call(key, meta, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class beginFileUpload_call extends org.apache.thrift.async.TAsyncMethodCall {
-      public beginFileUpload_call(org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+    public static class beginCreateBlob_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String key;
+      private SettableBlobMeta meta;
+      public beginCreateBlob_call(String key, SettableBlobMeta meta, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
+        this.key = key;
+        this.meta = meta;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("beginFileUpload", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        beginFileUpload_args args = new beginFileUpload_args();
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("beginCreateBlob", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        beginCreateBlob_args args = new beginCreateBlob_args();
+        args.set_key(key);
+        args.set_meta(meta);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public String getResult() throws AuthorizationException, org.apache.thrift.TException {
+      public String getResult() throws AuthorizationException, KeyAlreadyExistsException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_beginFileUpload();
+        return (new Client(prot)).recv_beginCreateBlob();
       }
     }
 
-    public void uploadChunk(String location, ByteBuffer chunk, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+    public void beginUpdateBlob(String key, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      uploadChunk_call method_call = new uploadChunk_call(location, chunk, resultHandler, this, ___protocolFactory, ___transport);
+      beginUpdateBlob_call method_call = new beginUpdateBlob_call(key, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class uploadChunk_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private String location;
-      private ByteBuffer chunk;
-      public uploadChunk_call(String location, ByteBuffer chunk, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+    public static class beginUpdateBlob_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String key;
+      public beginUpdateBlob_call(String key, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
-        this.location = location;
-        this.chunk = chunk;
+        this.key = key;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("uploadChunk", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        uploadChunk_args args = new uploadChunk_args();
-        args.set_location(location);
-        args.set_chunk(chunk);
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("beginUpdateBlob", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        beginUpdateBlob_args args = new beginUpdateBlob_args();
+        args.set_key(key);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public void getResult() throws AuthorizationException, org.apache.thrift.TException {
+      public String getResult() throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        (new Client(prot)).recv_uploadChunk();
+        return (new Client(prot)).recv_beginUpdateBlob();
       }
     }
 
-    public void finishFileUpload(String location, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+    public void uploadBlobChunk(String session, ByteBuffer chunk, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      finishFileUpload_call method_call = new finishFileUpload_call(location, resultHandler, this, ___protocolFactory, ___transport);
+      uploadBlobChunk_call method_call = new uploadBlobChunk_call(session, chunk, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class finishFileUpload_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private String location;
-      public finishFileUpload_call(String location, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+    public static class uploadBlobChunk_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String session;
+      private ByteBuffer chunk;
+      public uploadBlobChunk_call(String session, ByteBuffer chunk, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
-        this.location = location;
+        this.session = session;
+        this.chunk = chunk;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("finishFileUpload", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        finishFileUpload_args args = new finishFileUpload_args();
-        args.set_location(location);
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("uploadBlobChunk", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        uploadBlobChunk_args args = new uploadBlobChunk_args();
+        args.set_session(session);
+        args.set_chunk(chunk);
         args.write(prot);
         prot.writeMessageEnd();
       }
@@ -1551,813 +1981,939 @@ public class Nimbus {
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        (new Client(prot)).recv_finishFileUpload();
+        (new Client(prot)).recv_uploadBlobChunk();
       }
     }
 
-    public void beginFileDownload(String file, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+    public void finishBlobUpload(String session, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      beginFileDownload_call method_call = new beginFileDownload_call(file, resultHandler, this, ___protocolFactory, ___transport);
+      finishBlobUpload_call method_call = new finishBlobUpload_call(session, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class beginFileDownload_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private String file;
-      public beginFileDownload_call(String file, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+    public static class finishBlobUpload_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String session;
+      public finishBlobUpload_call(String session, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
-        this.file = file;
+        this.session = session;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("beginFileDownload", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        beginFileDownload_args args = new beginFileDownload_args();
-        args.set_file(file);
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("finishBlobUpload", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        finishBlobUpload_args args = new finishBlobUpload_args();
+        args.set_session(session);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public String getResult() throws AuthorizationException, org.apache.thrift.TException {
+      public void getResult() throws AuthorizationException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_beginFileDownload();
+        (new Client(prot)).recv_finishBlobUpload();
       }
     }
 
-    public void downloadChunk(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+    public void cancelBlobUpload(String session, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      downloadChunk_call method_call = new downloadChunk_call(id, resultHandler, this, ___protocolFactory, ___transport);
+      cancelBlobUpload_call method_call = new cancelBlobUpload_call(session, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class downloadChunk_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private String id;
-      public downloadChunk_call(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+    public static class cancelBlobUpload_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String session;
+      public cancelBlobUpload_call(String session, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
-        this.id = id;
+        this.session = session;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("downloadChunk", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        downloadChunk_args args = new downloadChunk_args();
-        args.set_id(id);
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("cancelBlobUpload", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        cancelBlobUpload_args args = new cancelBlobUpload_args();
+        args.set_session(session);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public ByteBuffer getResult() throws AuthorizationException, org.apache.thrift.TException {
+      public void getResult() throws AuthorizationException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_downloadChunk();
+        (new Client(prot)).recv_cancelBlobUpload();
       }
     }
 
-    public void getNimbusConf(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+    public void getBlobMeta(String key, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      getNimbusConf_call method_call = new getNimbusConf_call(resultHandler, this, ___protocolFactory, ___transport);
+      getBlobMeta_call method_call = new getBlobMeta_call(key, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class getNimbusConf_call extends org.apache.thrift.async.TAsyncMethodCall {
-      public getNimbusConf_call(org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+    public static class getBlobMeta_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String key;
+      public getBlobMeta_call(String key, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
+        this.key = key;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getNimbusConf", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getNimbusConf_args args = new getNimbusConf_args();
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getBlobMeta", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getBlobMeta_args args = new getBlobMeta_args();
+        args.set_key(key);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public String getResult() throws AuthorizationException, org.apache.thrift.TException {
+      public ReadableBlobMeta getResult() throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_getNimbusConf();
+        return (new Client(prot)).recv_getBlobMeta();
       }
     }
 
-    public void getClusterInfo(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+    public void setBlobMeta(String key, SettableBlobMeta meta, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      getClusterInfo_call method_call = new getClusterInfo_call(resultHandler, this, ___protocolFactory, ___transport);
+      setBlobMeta_call method_call = new setBlobMeta_call(key, meta, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class getClusterInfo_call extends org.apache.thrift.async.TAsyncMethodCall {
-      public getClusterInfo_call(org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+    public static class setBlobMeta_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String key;
+      private SettableBlobMeta meta;
+      public setBlobMeta_call(String key, SettableBlobMeta meta, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
+        this.key = key;
+        this.meta = meta;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getClusterInfo", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getClusterInfo_args args = new getClusterInfo_args();
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("setBlobMeta", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        setBlobMeta_args args = new setBlobMeta_args();
+        args.set_key(key);
+        args.set_meta(meta);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public ClusterSummary getResult() throws AuthorizationException, org.apache.thrift.TException {
+      public void getResult() throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_getClusterInfo();
+        (new Client(prot)).recv_setBlobMeta();
       }
     }
 
-    public void getTopologyInfo(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+    public void beginBlobDownload(String key, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      getTopologyInfo_call method_call = new getTopologyInfo_call(id, resultHandler, this, ___protocolFactory, ___transport);
+      beginBlobDownload_call method_call = new beginBlobDownload_call(key, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class getTopologyInfo_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private String id;
-      public getTopologyInfo_call(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+    public static class beginBlobDownload_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String key;
+      public beginBlobDownload_call(String key, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
-        this.id = id;
+        this.key = key;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTopologyInfo", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getTopologyInfo_args args = new getTopologyInfo_args();
-        args.set_id(id);
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("beginBlobDownload", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        beginBlobDownload_args args = new beginBlobDownload_args();
+        args.set_key(key);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public TopologyInfo getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException {
+      public BeginDownloadResult getResult() throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_getTopologyInfo();
+        return (new Client(prot)).recv_beginBlobDownload();
       }
     }
 
-    public void getTopologyInfoWithOpts(String id, GetInfoOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+    public void downloadBlobChunk(String session, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      getTopologyInfoWithOpts_call method_call = new getTopologyInfoWithOpts_call(id, options, resultHandler, this, ___protocolFactory, ___transport);
+      downloadBlobChunk_call method_call = new downloadBlobChunk_call(session, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class getTopologyInfoWithOpts_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private String id;
-      private GetInfoOptions options;
-      public getTopologyInfoWithOpts_call(String id, GetInfoOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+    public static class downloadBlobChunk_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String session;
+      public downloadBlobChunk_call(String session, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
-        this.id = id;
-        this.options = options;
+        this.session = session;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTopologyInfoWithOpts", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getTopologyInfoWithOpts_args args = new getTopologyInfoWithOpts_args();
-        args.set_id(id);
-        args.set_options(options);
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("downloadBlobChunk", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        downloadBlobChunk_args args = new downloadBlobChunk_args();
+        args.set_session(session);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public TopologyInfo getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException {
+      public ByteBuffer getResult() throws AuthorizationException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_getTopologyInfoWithOpts();
+        return (new Client(prot)).recv_downloadBlobChunk();
       }
     }
 
-    public void getTopologyPageInfo(String id, String window, boolean is_include_sys, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+    public void deleteBlob(String key, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      getTopologyPageInfo_call method_call = new getTopologyPageInfo_call(id, window, is_include_sys, resultHandler, this, ___protocolFactory, ___transport);
+      deleteBlob_call method_call = new deleteBlob_call(key, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class getTopologyPageInfo_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private String id;
-      private String window;
-      private boolean is_include_sys;
-      public getTopologyPageInfo_call(String id, String window, boolean is_include_sys, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+    public static class deleteBlob_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String key;
+      public deleteBlob_call(String key, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
-        this.id = id;
-        this.window = window;
-        this.is_include_sys = is_include_sys;
+        this.key = key;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTopologyPageInfo", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getTopologyPageInfo_args args = new getTopologyPageInfo_args();
-        args.set_id(id);
-        args.set_window(window);
-        args.set_is_include_sys(is_include_sys);
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("deleteBlob", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        deleteBlob_args args = new deleteBlob_args();
+        args.set_key(key);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public TopologyPageInfo getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException {
+      public void getResult() throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_getTopologyPageInfo();
+        (new Client(prot)).recv_deleteBlob();
       }
     }
 
-    public void getComponentPageInfo(String topology_id, String component_id, String window, boolean is_include_sys, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+    public void listBlobs(String session, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      getComponentPageInfo_call method_call = new getComponentPageInfo_call(topology_id, component_id, window, is_include_sys, resultHandler, this, ___protocolFactory, ___transport);
+      listBlobs_call method_call = new listBlobs_call(session, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class getComponentPageInfo_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private String topology_id;
-      private String component_id;
-      private String window;
-      private boolean is_include_sys;
-      public getComponentPageInfo_call(String topology_id, String component_id, String window, boolean is_include_sys, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+    public static class listBlobs_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String session;
+      public listBlobs_call(String session, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
-        this.topology_id = topology_id;
-        this.component_id = component_id;
-        this.window = window;
-        this.is_include_sys = is_include_sys;
+        this.session = session;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getComponentPageInfo", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getComponentPageInfo_args args = new getComponentPageInfo_args();
-        args.set_topology_id(topology_id);
-        args.set_component_id(component_id);
-        args.set_window(window);
-        args.set_is_include_sys(is_include_sys);
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("listBlobs", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        listBlobs_args args = new listBlobs_args();
+        args.set_session(session);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public ComponentPageInfo getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException {
+      public ListBlobsResult getResult() throws org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_getComponentPageInfo();
+        return (new Client(prot)).recv_listBlobs();
       }
     }
 
-    public void getTopologyConf(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+    public void getBlobReplication(String key, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      getTopologyConf_call method_call = new getTopologyConf_call(id, resultHandler, this, ___protocolFactory, ___transport);
+      getBlobReplication_call method_call = new getBlobReplication_call(key, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class getTopologyConf_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private String id;
-      public getTopologyConf_call(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+    public static class getBlobReplication_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String key;
+      public getBlobReplication_call(String key, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
-        this.id = id;
+        this.key = key;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTopologyConf", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getTopologyConf_args args = new getTopologyConf_args();
-        args.set_id(id);
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getBlobReplication", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getBlobReplication_args args = new getBlobReplication_args();
+        args.set_key(key);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public String getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException {
+      public int getResult() throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_getTopologyConf();
+        return (new Client(prot)).recv_getBlobReplication();
       }
     }
 
-    public void getTopology(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+    public void updateBlobReplication(String key, int replication, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      getTopology_call method_call = new getTopology_call(id, resultHandler, this, ___protocolFactory, ___transport);
+      updateBlobReplication_call method_call = new updateBlobReplication_call(key, replication, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class getTopology_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private String id;
-      public getTopology_call(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+    public static class updateBlobReplication_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String key;
+      private int replication;
+      public updateBlobReplication_call(String key, int replication, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
-        this.id = id;
+        this.key = key;
+        this.replication = replication;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTopology", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getTopology_args args = new getTopology_args();
-        args.set_id(id);
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("updateBlobReplication", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        updateBlobReplication_args args = new updateBlobReplication_args();
+        args.set_key(key);
+        args.set_replication(replication);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public StormTopology getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException {
+      public int getResult() throws AuthorizationException, KeyNotFoundException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_getTopology();
+        return (new Client(prot)).recv_updateBlobReplication();
       }
     }
 
-    public void getUserTopology(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+    public void createStateInZookeeper(String key, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      getUserTopology_call method_call = new getUserTopology_call(id, resultHandler, this, ___protocolFactory, ___transport);
+      createStateInZookeeper_call method_call = new createStateInZookeeper_call(key, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class getUserTopology_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private String id;
-      public getUserTopology_call(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+    public static class createStateInZookeeper_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String key;
+      public createStateInZookeeper_call(String key, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
-        this.id = id;
+        this.key = key;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getUserTopology", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getUserTopology_args args = new getUserTopology_args();
-        args.set_id(id);
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("createStateInZookeeper", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        createStateInZookeeper_args args = new createStateInZookeeper_args();
+        args.set_key(key);
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public StormTopology getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException {
+      public void getResult() throws org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_getUserTopology();
+        (new Client(prot)).recv_createStateInZookeeper();
       }
     }
 
-    public void getTopologyHistory(String user, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+    public void beginFileUpload(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      getTopologyHistory_call method_call = new getTopologyHistory_call(user, resultHandler, this, ___protocolFactory, ___transport);
+      beginFileUpload_call method_call = new beginFileUpload_call(resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class getTopologyHistory_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private String user;
-      public getTopologyHistory_call(String user, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+    public static class beginFileUpload_call extends org.apache.thrift.async.TAsyncMethodCall {
+      public beginFileUpload_call(org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
-        this.user = user;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTopologyHistory", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getTopologyHistory_args args = new getTopologyHistory_args();
-        args.set_user(user);
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("beginFileUpload", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        beginFileUpload_args args = new beginFileUpload_args();
         args.write(prot);
         prot.writeMessageEnd();
       }
 
-      public TopologyHistoryInfo getResult() throws AuthorizationException, org.apache.thrift.TException {
+      public String getResult() throws AuthorizationException, org.apache.thrift.TException {
         if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
           throw new IllegalStateException("Method call not finished!");
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_getTopologyHistory();
+        return (new Client(prot)).recv_beginFileUpload();
       }
     }
 
-  }
-
-  public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
-    private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName());
-    public Processor(I iface) {
-      super(iface, getProcessMap(new HashMap<String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>>()));
-    }
-
-    protected Processor(I iface, Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
-      super(iface, getProcessMap(processMap));
-    }
-
-    private static <I extends Iface> Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> getProcessMap(Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
-      processMap.put("submitTopology", new submitTopology());
-      processMap.put("submitTopologyWithOpts", new submitTopologyWithOpts());
-      processMap.put("killTopology", new killTopology());
-      processMap.put("killTopologyWithOpts", new killTopologyWithOpts());
-      processMap.put("activate", new activate());
-      processMap.put("deactivate", new deactivate());
-      processMap.put("rebalance", new rebalance());
-      processMap.put("setLogConfig", new setLogConfig());
-      processMap.put("getLogConfig", new getLogConfig());
-      processMap.put("debug", new debug());
-      processMap.put("setWorkerProfiler", new setWorkerProfiler());
-      processMap.put("getComponentPendingProfileActions", new getComponentPendingProfileActions());
-      processMap.put("uploadNewCredentials", new uploadNewCredentials());
-      processMap.put("beginFileUpload", new beginFileUpload());
-      processMap.put("uploadChunk", new uploadChunk());
-      processMap.put("finishFileUpload", new finishFileUpload());
-      processMap.put("beginFileDownload", new beginFileDownload());
-      processMap.put("downloadChunk", new downloadChunk());
-      processMap.put("getNimbusConf", new getNimbusConf());
-      processMap.put("getClusterInfo", new getClusterInfo());
-      processMap.put("getTopologyInfo", new getTopologyInfo());
-      processMap.put("getTopologyInfoWithOpts", new getTopologyInfoWithOpts());
-      processMap.put("getTopologyPageInfo", new getTopologyPageInfo());
-      processMap.put("getComponentPageInfo", new getComponentPageInfo());
-      processMap.put("getTopologyConf", new getTopologyConf());
-      processMap.put("getTopology", new getTopology());
-      processMap.put("getUserTopology", new getUserTopology());
-      processMap.put("getTopologyHistory", new getTopologyHistory());
-      return processMap;
+    public void uploadChunk(String location, ByteBuffer chunk, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      uploadChunk_call method_call = new uploadChunk_call(location, chunk, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
     }
 
-    public static class submitTopology<I extends Iface> extends org.apache.thrift.ProcessFunction<I, submitTopology_args> {
-      public submitTopology() {
-        super("submitTopology");
-      }
-
-      public submitTopology_args getEmptyArgsInstance() {
-        return new submitTopology_args();
+    public static class uploadChunk_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String location;
+      private ByteBuffer chunk;
+      public uploadChunk_call(String location, ByteBuffer chunk, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.location = location;
+        this.chunk = chunk;
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("uploadChunk", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        uploadChunk_args args = new uploadChunk_args();
+        args.set_location(location);
+        args.set_chunk(chunk);
+        args.write(prot);
+        prot.writeMessageEnd();
       }
 
-      public submitTopology_result getResult(I iface, submitTopology_args args) throws org.apache.thrift.TException {
-        submitTopology_result result = new submitTopology_result();
-        try {
-          iface.submitTopology(args.name, args.uploadedJarLocation, args.jsonConf, args.topology);
-        } catch (AlreadyAliveException e) {
-          result.e = e;
-        } catch (InvalidTopologyException ite) {
-          result.ite = ite;
-        } catch (AuthorizationException aze) {
-          result.aze = aze;
+      public void getResult() throws AuthorizationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
         }
-        return result;
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_uploadChunk();
       }
     }
 
-    public static class submitTopologyWithOpts<I extends Iface> extends org.apache.thrift.ProcessFunction<I, submitTopologyWithOpts_args> {
-      public submitTopologyWithOpts() {
-        super("submitTopologyWithOpts");
-      }
-
-      public submitTopologyWithOpts_args getEmptyArgsInstance() {
-        return new submitTopologyWithOpts_args();
+    public void finishFileUpload(String location, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      finishFileUpload_call method_call = new finishFileUpload_call(location, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class finishFileUpload_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String location;
+      public finishFileUpload_call(String location, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.location = location;
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("finishFileUpload", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        finishFileUpload_args args = new finishFileUpload_args();
+        args.set_location(location);
+        args.write(prot);
+        prot.writeMessageEnd();
       }
 
-      public submitTopologyWithOpts_result getResult(I iface, submitTopologyWithOpts_args args) throws org.apache.thrift.TException {
-        submitTopologyWithOpts_result result = new submitTopologyWithOpts_result();
-        try {
-          iface.submitTopologyWithOpts(args.name, args.uploadedJarLocation, args.jsonConf, args.topology, args.options);
-        } catch (AlreadyAliveException e) {
-          result.e = e;
-        } catch (InvalidTopologyException ite) {
-          result.ite = ite;
-        } catch (AuthorizationException aze) {
-          result.aze = aze;
+      public void getResult() throws AuthorizationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
         }
-        return result;
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_finishFileUpload();
       }
     }
 
-    public static class killTopology<I extends Iface> extends org.apache.thrift.ProcessFunction<I, killTopology_args> {
-      public killTopology() {
-        super("killTopology");
-      }
+    public void beginFileDownload(String file, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      beginFileDownload_call method_call = new beginFileDownload_call(file, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
 
-      public killTopology_args getEmptyArgsInstance() {
-        return new killTopology_args();
+    public static class beginFileDownload_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String file;
+      public beginFileDownload_call(String file, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.file = file;
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("beginFileDownload", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        beginFileDownload_args args = new beginFileDownload_args();
+        args.set_file(file);
+        args.write(prot);
+        prot.writeMessageEnd();
       }
 
-      public killTopology_result getResult(I iface, killTopology_args args) throws org.apache.thrift.TException {
-        killTopology_result result = new killTopology_result();
-        try {
-          iface.killTopology(args.name);
-        } catch (NotAliveException e) {
-          result.e = e;
-        } catch (AuthorizationException aze) {
-          result.aze = aze;
+      public String getResult() throws AuthorizationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
         }
-        return result;
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_beginFileDownload();
       }
     }
 
-    public static class killTopologyWithOpts<I extends Iface> extends org.apache.thrift.ProcessFunction<I, killTopologyWithOpts_args> {
-      public killTopologyWithOpts() {
-        super("killTopologyWithOpts");
-      }
+    public void downloadChunk(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      downloadChunk_call method_call = new downloadChunk_call(id, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
 
-      public killTopologyWithOpts_args getEmptyArgsInstance() {
-        return new killTopologyWithOpts_args();
+    public static class downloadChunk_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String id;
+      public downloadChunk_call(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.id = id;
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("downloadChunk", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        downloadChunk_args args = new downloadChunk_args();
+        args.set_id(id);
+        args.write(prot);
+        prot.writeMessageEnd();
       }
 
-      public killTopologyWithOpts_result getResult(I iface, killTopologyWithOpts_args args) throws org.apache.thrift.TException {
-        killTopologyWithOpts_result result = new killTopologyWithOpts_result();
-        try {
-          iface.killTopologyWithOpts(args.name, args.options);
-        } catch (NotAliveException e) {
-          result.e = e;
-        } catch (AuthorizationException aze) {
-          result.aze = aze;
+      public ByteBuffer getResult() throws AuthorizationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
         }
-        return result;
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_downloadChunk();
       }
     }
 
-    public static class activate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, activate_args> {
-      public activate() {
-        super("activate");
-      }
+    public void getNimbusConf(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getNimbusConf_call method_call = new getNimbusConf_call(resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
 
-      public activate_args getEmptyArgsInstance() {
-        return new activate_args();
+    public static class getNimbusConf_call extends org.apache.thrift.async.TAsyncMethodCall {
+      public getNimbusConf_call(org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getNimbusConf", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getNimbusConf_args args = new getNimbusConf_args();
+        args.write(prot);
+        prot.writeMessageEnd();
       }
 
-      public activate_result getResult(I iface, activate_args args) throws org.apache.thrift.TException {
-        activate_result result = new activate_result();
-        try {
-          iface.activate(args.name);
-        } catch (NotAliveException e) {
-          result.e = e;
-        } catch (AuthorizationException aze) {
-          result.aze = aze;
+      public String getResult() throws AuthorizationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
         }
-        return result;
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getNimbusConf();
       }
     }
 
-    public static class deactivate<I extends Iface> extends org.apache.thrift.ProcessFunction<I, deactivate_args> {
-      public deactivate() {
-        super("deactivate");
-      }
+    public void getClusterInfo(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getClusterInfo_call method_call = new getClusterInfo_call(resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
 
-      public deactivate_args getEmptyArgsInstance() {
-        return new deactivate_args();
+    public static class getClusterInfo_call extends org.apache.thrift.async.TAsyncMethodCall {
+      public getClusterInfo_call(org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getClusterInfo", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getClusterInfo_args args = new getClusterInfo_args();
+        args.write(prot);
+        prot.writeMessageEnd();
       }
 
-      public deactivate_result getResult(I iface, deactivate_args args) throws org.apache.thrift.TException {
-        deactivate_result result = new deactivate_result();
-        try {
-          iface.deactivate(args.name);
-        } catch (NotAliveException e) {
-          result.e = e;
-        } catch (AuthorizationException aze) {
-          result.aze = aze;
+      public ClusterSummary getResult() throws AuthorizationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
         }
-        return result;
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getClusterInfo();
       }
     }
 
-    public static class rebalance<I extends Iface> extends org.apache.thrift.ProcessFunction<I, rebalance_args> {
-      public rebalance() {
-        super("rebalance");
-      }
+    public void getTopologyInfo(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getTopologyInfo_call method_call = new getTopologyInfo_call(id, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
 
-      public rebalance_args getEmptyArgsInstance() {
-        return new rebalance_args();
+    public static class getTopologyInfo_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String id;
+      public getTopologyInfo_call(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.id = id;
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTopologyInfo", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getTopologyInfo_args args = new getTopologyInfo_args();
+        args.set_id(id);
+        args.write(prot);
+        prot.writeMessageEnd();
       }
 
-      public rebalance_result getResult(I iface, rebalance_args args) throws org.apache.thrift.TException {
-        rebalance_result result = new rebalance_result();
-        try {
-          iface.rebalance(args.name, args.options);
-        } catch (NotAliveException e) {
-          result.e = e;
-        } catch (InvalidTopologyException ite) {
-          result.ite = ite;
-        } catch (AuthorizationException aze) {
-          result.aze = aze;
+      public TopologyInfo getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
         }
-        return result;
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getTopologyInfo();
       }
     }
 
-    public static class setLogConfig<I extends Iface> extends org.apache.thrift.ProcessFunction<I, setLogConfig_args> {
-      public setLogConfig() {
-        super("setLogConfig");
-      }
+    public void getTopologyInfoWithOpts(String id, GetInfoOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getTopologyInfoWithOpts_call method_call = new getTopologyInfoWithOpts_call(id, options, resultHandler, this, ___protocolFactory, 

<TRUNCATED>

[15/17] storm git commit: removed a merge issue multiple permission setting

Posted by bo...@apache.org.
removed a merge issue multiple permission setting


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

Branch: refs/heads/master
Commit: 51058027a3d77145adef244c4cf66b5241fd1252
Parents: 6687bed
Author: Sanket <sc...@untilservice-lm>
Authored: Tue Dec 1 15:15:36 2015 -0600
Committer: Sanket <sc...@untilservice-lm>
Committed: Tue Dec 1 15:15:36 2015 -0600

----------------------------------------------------------------------
 .../main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java  | 2 +-
 storm-core/src/clj/backtype/storm/daemon/supervisor.clj           | 3 +--
 2 files changed, 2 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/51058027/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java
----------------------------------------------------------------------
diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java
index c344bcf..fbc277e 100644
--- a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java
+++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java
@@ -65,7 +65,7 @@ import static backtype.storm.blobstore.BlobStoreAclHandler.WRITE;
  * 1. The USER interacts with nimbus to upload and access blobs through NimbusBlobStore Client API. Here, unlike
  * local blob store which stores the blobs locally, the nimbus talks to HDFS to upload the blobs.
  * 2. The USER sets the ACLs, and the blob access is validated against these ACLs.
- * 3. The SUPERVISOR interacts with nimbus thorugh HdfsClientBlobStore to download the blobs. Here, unlike local
+ * 3. The SUPERVISOR interacts with nimbus through HdfsClientBlobStore to download the blobs. Here, unlike local
  * blob store the supervisor interacts with HDFS directly to download the blobs. The call to HdfsBlobStore is made as a "null"
  * subject. The blobstore gets the hadoop user and validates permissions for the supervisor.
  */

http://git-wip-us.apache.org/repos/asf/storm/blob/51058027/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
index e066269..d85893b 100644
--- a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
@@ -869,8 +869,7 @@
         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)))
+        (FileUtils/forceMkdir user-dir))
       (try
         (let [localized-resources (.getBlobs localizer localresources user topo-name user-dir)]
           (setup-blob-permission conf storm-conf (.toString user-dir))


[13/17] storm git commit: do not cache subject as it calls thread local req context

Posted by bo...@apache.org.
do not cache subject as it calls thread local req context


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

Branch: refs/heads/master
Commit: 57240b4f1c1aafce51e9465af9b87b7f41e88dec
Parents: 7029aee
Author: Sanket <sc...@untilservice-lm>
Authored: Mon Nov 30 15:29:26 2015 -0600
Committer: Sanket <sc...@untilservice-lm>
Committed: Mon Nov 30 15:29:26 2015 -0600

----------------------------------------------------------------------
 .../src/clj/backtype/storm/daemon/nimbus.clj    | 20 +++++++++-----------
 1 file changed, 9 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/57240b4f/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index a53ff82..9834f25 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -205,17 +205,15 @@
 (defn inbox [nimbus]
   (master-inbox (:conf nimbus)))
 
-(defn- get-subject []
+(defn- get-subject
+  []
   (let [req (ReqContext/context)]
     (.subject req)))
 
-(def user-subject
-  (get-subject))
-
 (defn- read-storm-conf [conf storm-id blob-store]
   (clojurify-structure
     (Utils/fromCompressedJsonConf
-      (.readBlob blob-store (master-stormconf-key storm-id) user-subject))))
+      (.readBlob blob-store (master-stormconf-key storm-id) (get-subject)))))
 
 (declare delay-event)
 (declare mk-assignments)
@@ -456,7 +454,7 @@
     (iterator-seq key-iter)))
 
 (defn- setup-storm-code [nimbus conf storm-id tmp-jar-location storm-conf topology]
-  (let [subject user-subject
+  (let [subject (get-subject)
         storm-cluster-state (:storm-cluster-state nimbus)
         blob-store (:blob-store nimbus)
         jar-key (master-stormjar-key storm-id)
@@ -476,7 +474,7 @@
 
 (defn- read-storm-topology [storm-id blob-store]
   (Utils/deserialize
-    (.readBlob blob-store (master-stormcode-key storm-id) user-subject) StormTopology))
+    (.readBlob blob-store (master-stormcode-key storm-id) (get-subject)) StormTopology))
 
 (defn get-blob-replication-count
   [blob-key nimbus]
@@ -1916,14 +1914,14 @@
         (let [session-id (uuid)]
           (.put (:blob-uploaders nimbus)
             session-id
-            (.createBlob (:blob-store nimbus) blob-key blob-meta user-subject))
+            (.createBlob (:blob-store nimbus) blob-key blob-meta (get-subject)))
           (log-message "Created blob for " blob-key
             " with session id " session-id)
           (str session-id)))
 
       (^String beginUpdateBlob [this ^String blob-key]
         (let [^AtomicOutputStream os (.updateBlob (:blob-store nimbus)
-                                       blob-key user-subject)]
+                                       blob-key (get-subject))]
           (let [session-id (uuid)]
             (.put (:blob-uploaders nimbus) session-id os)
             (log-message "Created upload session for " blob-key
@@ -1978,7 +1976,7 @@
 
       (^ReadableBlobMeta getBlobMeta [this ^String blob-key]
         (let [^ReadableBlobMeta ret (.getBlobMeta (:blob-store nimbus)
-                                      blob-key user-subject)]
+                                      blob-key (get-subject))]
           ret))
 
       (^void setBlobMeta [this ^String blob-key ^SettableBlobMeta blob-meta]
@@ -1988,7 +1986,7 @@
 
       (^BeginDownloadResult beginBlobDownload [this ^String blob-key]
         (let [^InputStreamWithMeta is (.getBlob (:blob-store nimbus)
-                                        blob-key user-subject)]
+                                        blob-key (get-subject))]
           (let [session-id (uuid)
                 ret (BeginDownloadResult. (.getVersion is) (str session-id))]
             (.set_data_size ret (.getFileLength is))


[11/17] storm git commit: Blobstore API STORM- 876

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 0da88b0..f093ce5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -175,6 +175,7 @@
         <java_jmx.version>0.3.1</java_jmx.version>
         <compojure.version>1.1.3</compojure.version>
         <hiccup.version>0.3.6</hiccup.version>
+        <commons-compress.version>1.4.1</commons-compress.version>
         <commons-io.version>2.4</commons-io.version>
         <commons-lang.version>2.5</commons-lang.version>
         <commons-exec.version>1.1</commons-exec.version>
@@ -355,6 +356,11 @@
             </dependency>
             <dependency>
                 <groupId>org.apache.commons</groupId>
+                <artifactId>commons-compress</artifactId>
+                <version>${commons-compress.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.commons</groupId>
                 <artifactId>commons-exec</artifactId>
                 <version>${commons-exec.version}</version>
             </dependency>
@@ -469,7 +475,12 @@
                 <artifactId>curator-client</artifactId>
                 <version>${curator.version}</version>
             </dependency>
-
+            <dependency>
+                <groupId>org.apache.curator</groupId>
+                <artifactId>curator-test</artifactId>
+                <version>${curator.version}</version>
+                <scope>test</scope>
+            </dependency>
             <dependency>
                 <groupId>com.googlecode.json-simple</groupId>
                 <artifactId>json-simple</artifactId>
@@ -659,6 +670,7 @@
                 <version>${thrift.version}</version>
                 <scope>compile</scope>
             </dependency>
+			<!-- used by examples/storm-starter -->
             <dependency>
                 <groupId>junit</groupId>
                 <artifactId>junit</artifactId>
@@ -818,6 +830,8 @@
                         <exclude>**/.idea/**</exclude>
                         <!-- module specific testing artifacts -->
                         <exclude>**/metastore_db/**</exclude>
+                        <!-- anything written into build should be ignored -->
+                        <exclude>**/build/**</exclude>
 
                         <!-- exclude CHANGELOG, VERSION, AND TODO files -->
                         <exclude>**/CHANGELOG.md</exclude>

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/pom.xml
----------------------------------------------------------------------
diff --git a/storm-core/pom.xml b/storm-core/pom.xml
index 083cdca..72c4a3a 100644
--- a/storm-core/pom.xml
+++ b/storm-core/pom.xml
@@ -36,6 +36,17 @@
     </properties>
 
     <dependencies>
+        <!--Hadoop Mini Cluster cannot use log4j2 bridge,
+            Surefire has a way to exclude the conflicting log4j API jar
+            from the classpath, classpathDependencyExcludes, but it didn't work in practice.
+            This is here as a work around to place it at the beginning of the classpath
+            even though maven does not officially support ordering of the classpath.-->
+        <dependency>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+            <version>1.2.17</version>
+            <scope>test</scope>
+        </dependency>
         <dependency>
             <groupId>com.esotericsoftware.kryo</groupId>
             <artifactId>kryo</artifactId>
@@ -140,6 +151,10 @@
         </dependency>
         <dependency>
             <groupId>org.apache.commons</groupId>
+            <artifactId>commons-compress</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
             <artifactId>commons-exec</artifactId>
             <scope>compile</scope>
         </dependency>
@@ -178,7 +193,6 @@
                 </exclusion>
             </exclusions>
         </dependency>
-
         <dependency>
             <groupId>org.apache.curator</groupId>
             <artifactId>curator-recipes</artifactId>
@@ -193,7 +207,20 @@
                 </exclusion>
             </exclusions>
         </dependency>
-
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-test</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
         <dependency>
             <groupId>com.googlecode.json-simple</groupId>
             <artifactId>json-simple</artifactId>
@@ -433,6 +460,7 @@
                             <include>org.yaml:snakeyaml</include>
                             <include>org.jgrapht:jgrapht-core</include>
                             <include>org.apache.commons:commons-exec</include>
+                            <include>org.apache.commons:commons-compress</include>
                             <include>commons-io:commons-io</include>
                             <include>commons-codec:commons-codec</include>
                             <include>commons-fileupload:commons-fileupload</include>
@@ -574,6 +602,10 @@
                           <shadedPattern>org.apache.storm.shade.org.apache.commons.io</shadedPattern>
                         </relocation>
                         <relocation>
+                            <pattern>org.apache.commons.compress</pattern>
+                            <shadedPattern>org.apache.storm.shade.org.apache.commons.compress</shadedPattern>
+                        </relocation>
+                        <relocation>
                           <pattern>org.apache.commons.codec</pattern>
                           <shadedPattern>org.apache.storm.shade.org.apache.commons.codec</shadedPattern>
                         </relocation>

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/clj/backtype/storm/blobstore.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/blobstore.clj b/storm-core/src/clj/backtype/storm/blobstore.clj
new file mode 100644
index 0000000..936f4b5
--- /dev/null
+++ b/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)))))

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/clj/backtype/storm/cluster.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/cluster.clj b/storm-core/src/clj/backtype/storm/cluster.clj
index 35aa8c8..ebe4955 100644
--- a/storm-core/src/clj/backtype/storm/cluster.clj
+++ b/storm-core/src/clj/backtype/storm/cluster.clj
@@ -45,18 +45,14 @@
     (log-debug "Creating cluster state: " (.toString clazz))
     (or (.mkState state-instance conf auth-conf acls context)
         nil)))
-  
 
 (defprotocol StormClusterState
   (assignments [this callback])
   (assignment-info [this storm-id callback])
   (assignment-info-with-version [this storm-id callback])
   (assignment-version [this storm-id callback])
-  ;returns topologyIds under /stormroot/code-distributor
-  (code-distributor [this callback])
-  ;returns lits of nimbusinfos under /stormroot/code-distributor/storm-id
-  (code-distributor-info [this storm-id])
-
+  ;returns key information under /storm/blobstore/key
+  (blobstore-info [this blob-key])
   ;returns list of nimbus summaries stored under /stormroot/nimbuses/<nimbus-ids> -> <data>
   (nimbuses [this])
   ;adds the NimbusSummary to /stormroot/nimbuses/nimbus-id
@@ -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
+  ;; host:port and version info of the blob
+  (setup-blobstore! [this key nimbusInfo versionInfo])
+  (active-keys [this])
+  (blobstore [this callback])
   (remove-storm! [this storm-id])
+  (remove-blobstore-key! [this blob-key])
+  (remove-key-version! [this blob-key])
   (report-error [this storm-id component-id node port error])
   (errors [this storm-id component-id])
   (last-error [this storm-id component-id])
@@ -107,7 +108,9 @@
 (def WORKERBEATS-ROOT "workerbeats")
 (def BACKPRESSURE-ROOT "backpressure")
 (def ERRORS-ROOT "errors")
-(def CODE-DISTRIBUTOR-ROOT "code-distributor")
+(def BLOBSTORE-ROOT "blobstore")
+; Stores the latest update sequence for a blob
+(def BLOBSTORE-MAX-KEY-SEQUENCE-NUMBER-ROOT "blobstoremaxkeysequencenumber")
 (def NIMBUSES-ROOT "nimbuses")
 (def CREDENTIALS-ROOT "credentials")
 (def LOGCONFIG-ROOT "logconfigs")
@@ -119,7 +122,9 @@
 (def WORKERBEATS-SUBTREE (str "/" WORKERBEATS-ROOT))
 (def BACKPRESSURE-SUBTREE (str "/" BACKPRESSURE-ROOT))
 (def ERRORS-SUBTREE (str "/" ERRORS-ROOT))
-(def CODE-DISTRIBUTOR-SUBTREE (str "/" CODE-DISTRIBUTOR-ROOT))
+;; Blobstore subtree /storm/blobstore
+(def BLOBSTORE-SUBTREE (str "/" BLOBSTORE-ROOT))
+(def BLOBSTORE-MAX-KEY-SEQUENCE-NUMBER-SUBTREE (str "/" BLOBSTORE-MAX-KEY-SEQUENCE-NUMBER-ROOT))
 (def NIMBUSES-SUBTREE (str "/" NIMBUSES-ROOT))
 (def CREDENTIALS-SUBTREE (str "/" CREDENTIALS-ROOT))
 (def LOGCONFIG-SUBTREE (str "/" LOGCONFIG-ROOT))
@@ -133,9 +138,13 @@
   [id]
   (str ASSIGNMENTS-SUBTREE "/" id))
 
-(defn code-distributor-path
-  [id]
-  (str CODE-DISTRIBUTOR-SUBTREE "/" id))
+(defn blobstore-path
+  [key]
+  (str BLOBSTORE-SUBTREE "/" key))
+
+(defn blobstore-max-key-sequence-number-path
+  [key]
+  (str BLOBSTORE-MAX-KEY-SEQUENCE-NUMBER-SUBTREE "/" key))
 
 (defn nimbus-path
   [id]
@@ -244,7 +253,7 @@
         backpressure-callback (atom {})   ;; we want to reigister a topo directory getChildren callback for all workers of this dir
         assignments-callback (atom nil)
         storm-base-callback (atom {})
-        code-distributor-callback (atom nil)
+        blobstore-callback (atom nil)
         credentials-callback (atom {})
         log-config-callback (atom {})
         state-id (.register
@@ -259,14 +268,14 @@
                                                (issue-map-callback! assignment-version-callback (first args))
                                                (issue-map-callback! assignment-info-with-version-callback (first args))))
                          SUPERVISORS-ROOT (issue-callback! supervisors-callback)
-                         CODE-DISTRIBUTOR-ROOT (issue-callback! code-distributor-callback)
+                         BLOBSTORE-ROOT (issue-callback! blobstore-callback) ;; callback register for blobstore
                          STORMS-ROOT (issue-map-callback! storm-base-callback (first args))
                          CREDENTIALS-ROOT (issue-map-callback! credentials-callback (first args))
                          LOGCONFIG-ROOT (issue-map-callback! log-config-callback (first args))
                          BACKPRESSURE-ROOT (issue-map-callback! backpressure-callback (first args))
                          ;; this should never happen
                          (exit-process! 30 "Unknown callback for subtree " subtree args)))))]
-    (doseq [p [ASSIGNMENTS-SUBTREE STORMS-SUBTREE SUPERVISORS-SUBTREE WORKERBEATS-SUBTREE ERRORS-SUBTREE CODE-DISTRIBUTOR-SUBTREE NIMBUSES-SUBTREE
+    (doseq [p [ASSIGNMENTS-SUBTREE STORMS-SUBTREE SUPERVISORS-SUBTREE WORKERBEATS-SUBTREE ERRORS-SUBTREE BLOBSTORE-SUBTREE NIMBUSES-SUBTREE
                LOGCONFIG-SUBTREE]]
       (.mkdirs cluster-state p acls))
     (reify
@@ -299,13 +308,13 @@
           (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))
-        (do
-          (.sync_path cluster-state CODE-DISTRIBUTOR-SUBTREE)
-          (.get_children cluster-state CODE-DISTRIBUTOR-SUBTREE (not-nil? callback))))
+          (reset! blobstore-callback callback))
+        (.sync_path cluster-state BLOBSTORE-SUBTREE)
+        (.get_children cluster-state BLOBSTORE-SUBTREE (not-nil? callback)))
 
       (nimbuses
         [this]
@@ -327,18 +336,29 @@
         
         (.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)))))
+      (setup-blobstore!
+        [this key nimbusInfo versionInfo]
+        (let [path (str (blobstore-path key) "/" (.toHostPortString nimbusInfo) "-" versionInfo)]
+          (log-message "setup-path" path)
+          (.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)))
+
+      (blobstore-info
+        [this blob-key]
+        (let [path (blobstore-path blob-key)]
+          (.sync_path cluster-state path)
+          (.get_children cluster-state path false)))
 
       (active-storms
         [this]
         (.get_children cluster-state STORMS-SUBTREE false))
 
+      (active-keys
+        [this]
+        (.get_children cluster-state BLOBSTORE-SUBTREE false))
+
       (heartbeat-storms
         [this]
         (.get_worker_hb_children cluster-state WORKERBEATS-SUBTREE false))
@@ -526,18 +546,18 @@
         (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)))
+      (remove-blobstore-key!
+        [this blob-key]
+        (log-debug "removing key" blob-key)
+        (.delete_node cluster-state (blobstore-path blob-key)))
+
+      (remove-key-version!
+        [this blob-key]
+        (.delete_node cluster-state (blobstore-max-key-sequence-number-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))
         (.delete_node cluster-state (credentials-path storm-id))
         (.delete_node cluster-state (log-config-path storm-id))
         (.delete_node cluster-state (profiler-config-path storm-id))

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/clj/backtype/storm/cluster_state/zookeeper_state_factory.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/cluster_state/zookeeper_state_factory.clj b/storm-core/src/clj/backtype/storm/cluster_state/zookeeper_state_factory.clj
index ff942db..fa36240 100644
--- a/storm-core/src/clj/backtype/storm/cluster_state/zookeeper_state_factory.clj
+++ b/storm-core/src/clj/backtype/storm/cluster_state/zookeeper_state_factory.clj
@@ -149,6 +149,10 @@
        [this path]
        (zk/sync-path zk-writer path))
 
+      (delete-node-blobstore
+        [this path nimbus-host-port-info]
+        (zk/delete-node-blobstore zk-writer path nimbus-host-port-info))
+
      (close
        [this]
        (reset! active false)

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/clj/backtype/storm/command/blobstore.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/command/blobstore.clj b/storm-core/src/clj/backtype/storm/command/blobstore.clj
new file mode 100644
index 0000000..ae7f919
--- /dev/null
+++ b/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]
+           [backtype.storm.generated SettableBlobMeta AccessControl AuthorizationException
+            KeyNotFoundException]
+           [backtype.storm.blobstore BlobStoreAclHandler])
+  (:use [backtype.storm config]
+        [clojure.string :only [split]]
+        [clojure.tools.cli :only [cli]]
+        [clojure.java.io :only [copy input-stream output-stream]]
+        [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-factor} [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))]
+    (validate-key-name! key)
+    (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-error "ACCESS DENIED to key: " key)))
+          (catch KeyNotFoundException knf
+            (if-not (empty? args) (log-error 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)]
+                 (log-message "Current replication factor " blob-replication)
+                 blob-replication)
+      "--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)]
+                       (log-message "Replication factor is set to " blob-replication)
+                       blob-replication)))
+      :else (throw (RuntimeException. (str sub-command " is not a supported blobstore command")))))))
+
+(defn -main [& args]
+  (let [command (first args)
+        new-args (rest args)]
+    (condp = command
+      "cat" (read-cli new-args)
+      "create" (create-cli new-args)
+      "update" (update-cli new-args)
+      "delete" (delete-cli new-args)
+      "list" (list-cli new-args)
+      "set-acl" (set-acl-cli new-args)
+      "replication" (rep-cli new-args)
+      :else (throw (RuntimeException. (str command " is not a supported blobstore command"))))))

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/clj/backtype/storm/config.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/config.clj b/storm-core/src/clj/backtype/storm/config.clj
index 94b66c3..1617a3b 100644
--- a/storm-core/src/clj/backtype/storm/config.clj
+++ b/storm-core/src/clj/backtype/storm/config.clj
@@ -107,6 +107,18 @@
     (FileUtils/forceMkdir (File. ret))
     ret))
 
+(defn master-stormjar-key
+  [topology-id]
+  (str topology-id "-stormjar.jar"))
+
+(defn master-stormcode-key
+  [topology-id]
+  (str topology-id "-stormcode.ser"))
+
+(defn master-stormconf-key
+  [topology-id]
+  (str topology-id "-stormconf.ser"))
+
 (defn master-stormdist-root
   ([conf]
    (str (master-local-dir conf) file-path-separator "stormdist"))
@@ -119,6 +131,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))))))
+
 (defn master-storm-metafile-path [stormroot ]
   (str stormroot file-path-separator "storm-code-distributor.meta"))
 
@@ -197,7 +213,7 @@
   (let [stormroot (supervisor-stormdist-root conf storm-id)
         conf-path (supervisor-stormconf-path stormroot)
         topology-path (supervisor-stormcode-path stormroot)]
-    (merge conf (clojurify-structure (Utils/fromCompressedJsonConf (FileUtils/readFileToByteArray (File. conf-path)))))))
+    (read-supervisor-storm-conf-given-path conf conf-path)))
 
 (defn read-supervisor-topology
   [conf storm-id]
@@ -221,7 +237,11 @@
     nil
     )))
 
-  
+(defn get-id-from-blob-key
+  [key]
+  (if-let [groups (re-find #"^(.*)((-stormjar\.jar)|(-stormcode\.ser)|(-stormconf\.ser))$" key)]
+    (nth groups 1)))
+
 (defn set-worker-user! [conf worker-id user]
   (log-message "SET worker-user " worker-id " " user)
   (let [file (worker-user-file conf worker-id)]

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index 71d4654..a53ff82 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -15,14 +15,22 @@
 ;; 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.nio ByteBuffer]
+           [java.util Collections List HashMap ArrayList Iterator])
+  (:import [backtype.storm.blobstore AtomicOutputStream BlobStoreAclHandler
+            InputStreamWithMeta KeyFilter KeySequenceNumber BlobSynchronizer])
+  (:import [java.io File FileOutputStream FileInputStream])
   (:import [java.net InetAddress])
   (:import [java.nio.channels Channels WritableByteChannel])
   (:import [backtype.storm.security.auth ThriftServer ThriftConnectionType ReqContext AuthUtils])
@@ -31,12 +39,12 @@
             Cluster Topologies SchedulerAssignment SchedulerAssignmentImpl DefaultScheduler ExecutorDetails])
   (:import [backtype.storm.nimbus NimbusInfo])
   (:import [backtype.storm.utils TimeCacheMap TimeCacheMap$ExpiredCallback Utils ThriftTopologyUtils
-            BufferFileInputStream])
+            BufferFileInputStream BufferInputStream])
   (:import [backtype.storm.generated NotAliveException AlreadyAliveException StormTopology ErrorInfo
             ExecutorInfo InvalidTopologyException Nimbus$Iface Nimbus$Processor SubmitOptions TopologyInitialStatus
             KillOptions RebalanceOptions ClusterSummary SupervisorSummary TopologySummary TopologyInfo TopologyHistoryInfo
-            ExecutorSummary AuthorizationException GetInfoOptions NumErrorsChoice
-            ComponentPageInfo TopologyPageInfo LogConfig LogLevel LogLevelAction
+            ExecutorSummary AuthorizationException GetInfoOptions NumErrorsChoice SettableBlobMeta ReadableBlobMeta
+            BeginDownloadResult ListBlobsResult ComponentPageInfo TopologyPageInfo LogConfig LogLevel LogLevelAction
             ProfileRequest ProfileAction NodeInfo])
   (:import [backtype.storm.daemon Shutdownable])
   (:use [backtype.storm util config log timer zookeeper local-state])
@@ -47,6 +55,7 @@
   (:require [clojure.set :as set])
   (:import [backtype.storm.daemon.common StormBase Assignment])
   (:use [backtype.storm.daemon common])
+  (:use [backtype.storm config])
   (:import [org.apache.zookeeper data.ACL ZooDefs$Ids ZooDefs$Perms])
   (:import [backtype.storm.utils VersionInfo])
   (:require [clj-time.core :as time])
@@ -112,8 +121,7 @@
     scheduler
     ))
 
-(defmulti mk-code-distributor cluster-mode)
-(defmulti sync-code cluster-mode)
+(defmulti blob-sync cluster-mode)
 
 (defnk is-leader [nimbus :throw-exception true]
   (let [leader-elector (:leader-elector nimbus)]
@@ -126,6 +134,25 @@
   [(first ZooDefs$Ids/CREATOR_ALL_ACL) 
    (ACL. (bit-or ZooDefs$Perms/READ ZooDefs$Perms/CREATE) ZooDefs$Ids/ANYONE_ID_UNSAFE)])
 
+(defn mk-blob-cache-map
+  "Constructs a TimeCacheMap instance with a blob store timeout whose
+  expiration callback invokes cancel on the value held by an expired entry when
+  that value is an AtomicOutputStream and calls close otherwise."
+  [conf]
+  (TimeCacheMap.
+    (int (conf NIMBUS-BLOBSTORE-EXPIRATION-SECS))
+    (reify TimeCacheMap$ExpiredCallback
+      (expire [this id stream]
+        (if (instance? AtomicOutputStream stream)
+          (.cancel stream)
+          (.close stream))))))
+
+(defn mk-bloblist-cache-map
+  "Constructs a TimeCacheMap instance with a blobstore timeout and no callback
+  function."
+  [conf]
+  (TimeCacheMap. (int (conf NIMBUS-BLOBSTORE-EXPIRATION-SECS))))
+
 (defn create-tology-action-notifier [conf]
   (when-not (clojure.string/blank? (conf NIMBUS-TOPOLOGY-ACTION-NOTIFIER-PLUGIN))
     (let [instance (new-instance (conf NIMBUS-TOPOLOGY-ACTION-NOTIFIER-PLUGIN))]
@@ -153,6 +180,10 @@
      :heartbeats-cache (atom {})
      :downloaders (file-cache-map conf)
      :uploaders (file-cache-map conf)
+     :blob-store (Utils/getNimbusBlobStore conf (NimbusInfo/fromConf conf))
+     :blob-downloaders (mk-blob-cache-map conf)
+     :blob-uploaders (mk-blob-cache-map conf)
+     :blob-listers (mk-bloblist-cache-map conf)
      :uptime (uptime-computer)
      :validator (new-instance (conf NIMBUS-TOPOLOGY-VALIDATOR))
      :timer (mk-timer :kill-fn (fn [t]
@@ -161,7 +192,6 @@
                                  ))
      :scheduler (mk-scheduler conf inimbus)
      :leader-elector (zk-leader-elector conf)
-     :code-distributor (mk-code-distributor conf)
      :id->sched-status (atom {})
      :node-id->resources (atom {}) ;;resources of supervisors
      :id->resources (atom {}) ;;resources of topologies
@@ -175,22 +205,44 @@
 (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)))
+
+(def user-subject
+  (get-subject))
+
+(defn- read-storm-conf [conf storm-id blob-store]
+  (clojurify-structure
+    (Utils/fromCompressedJsonConf
+      (.readBlob blob-store (master-stormconf-key storm-id) user-subject))))
 
 (declare delay-event)
 (declare mk-assignments)
 
+(defn get-nimbus-subject
+  []
+  (let [subject (Subject.)
+        principal (NimbusPrincipal.)
+        principals (.getPrincipals subject)]
+    (.add principals principal)
+    subject))
+
+(def nimbus-subject
+  (get-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)]))
+
 (defn kill-transition [nimbus storm-id]
   (fn [kill-time]
     (let [delay (if kill-time
                   kill-time
-                  (get (read-storm-conf (:conf nimbus) storm-id)
+                  (get (read-storm-conf (:conf nimbus) storm-id (:blob-store nimbus))
                        TOPOLOGY-MESSAGE-TIMEOUT-SECS))]
       (delay-event nimbus
                    storm-id
@@ -205,7 +257,7 @@
   (fn [time num-workers executor-overrides]
     (let [delay (if time
                   time
-                  (get (read-storm-conf (:conf nimbus) storm-id)
+                  (get (read-storm-conf (:conf nimbus) storm-id (:blob-store nimbus))
                        TOPOLOGY-MESSAGE-TIMEOUT-SECS))]
       (delay-event nimbus
                    storm-id
@@ -250,6 +302,10 @@
                       (log-message "Killing topology: " storm-id)
                       (.remove-storm! (:storm-cluster-state nimbus)
                                       storm-id)
+                      (when (instance? LocalFsBlobStore (:blob-store nimbus))
+                        (doseq [blob-key (get-key-list-from-id (:conf nimbus) storm-id)]
+                          (.remove-blobstore-key! (:storm-cluster-state nimbus) blob-key)
+                          (.remove-key-version! (:storm-cluster-state nimbus) blob-key)))
                       nil)
             }
    :rebalancing {:startup (fn [] (delay-event nimbus
@@ -391,53 +447,99 @@
       [(.getNodeId slot) (.getPort slot)]
       )))
 
+(defn- get-version-for-key [key nimbus-host-port-info conf]
+  (let [version (KeySequenceNumber. key nimbus-host-port-info)]
+    (.getKeySequenceNumber version conf)))
+
+(defn get-key-seq-from-blob-store [blob-store]
+  (let [key-iter (.listKeys blob-store)]
+    (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 user-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) user-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 (neg? 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) nimbus)))
+        (reset! current-replication-count-code  (get-blob-replication-count (master-stormcode-key storm-id) nimbus))
+        (reset! current-replication-count-jar  (get-blob-replication-count (master-stormjar-key storm-id) nimbus))))
+    (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) nimbus-subject) StormTopology))
 
 (declare compute-executor->component)
 
+(defn read-storm-conf-as-nimbus [storm-id blob-store]
+  (clojurify-structure
+    (Utils/fromCompressedJsonConf
+      (.readBlob blob-store (master-stormconf-key storm-id) nimbus-subject))))
+
 (defn read-topology-details [nimbus storm-id]
   (let [conf (:conf nimbus)
+        blob-store (:blob-store nimbus)
         storm-base (.storm-base (:storm-cluster-state nimbus) storm-id nil)
-        topology-conf (read-storm-conf conf storm-id)
-        topology (read-storm-topology conf storm-id)
+        topology-conf (read-storm-conf-as-nimbus storm-id blob-store)
+        topology (read-storm-topology-as-nimbus storm-id blob-store)
         executor->component (->> (compute-executor->component nimbus storm-id)
                                  (map-key (fn [[start-task end-task]]
                                             (ExecutorDetails. (int start-task) (int end-task)))))]
@@ -530,10 +632,11 @@
 
 (defn- compute-executors [nimbus storm-id]
   (let [conf (:conf nimbus)
+        blob-store (:blob-store nimbus)
         storm-base (.storm-base (:storm-cluster-state nimbus) storm-id nil)
         component->executors (:component->executors storm-base)
-        storm-conf (read-storm-conf conf storm-id)
-        topology (read-storm-topology conf storm-id)
+        storm-conf (read-storm-conf-as-nimbus storm-id blob-store)
+        topology (read-storm-topology-as-nimbus storm-id blob-store)
         task->component (storm-task-info topology storm-conf)]
     (->> (storm-task-info topology storm-conf)
          reverse-map
@@ -546,9 +649,10 @@
 
 (defn- compute-executor->component [nimbus storm-id]
   (let [conf (:conf nimbus)
+        blob-store (:blob-store nimbus)
         executors (compute-executors nimbus storm-id)
-        topology (read-storm-topology conf storm-id)
-        storm-conf (read-storm-conf conf storm-id)
+        topology (read-storm-topology-as-nimbus storm-id blob-store)
+        storm-conf (read-storm-conf-as-nimbus storm-id blob-store)
         task->component (storm-task-info topology storm-conf)
         executor->component (into {} (for [executor executors
                                            :let [start-task (first executor)
@@ -838,7 +942,7 @@
                                                                         )))
                                               worker->resources (get new-assigned-worker->resources topology-id)]]
                                    {topology-id (Assignment.
-                                                 (master-stormdist-root conf topology-id)
+                                                 (conf STORM-LOCAL-DIR)
                                                  (select-keys all-node->host all-nodes)
                                                  executor->node+port
                                                  start-times
@@ -875,8 +979,9 @@
   {:pre [(#{:active :inactive} topology-initial-status)]}
   (let [storm-cluster-state (:storm-cluster-state nimbus)
         conf (:conf nimbus)
-        storm-conf (read-storm-conf conf storm-id)
-        topology (system-topology! storm-conf (read-storm-topology conf storm-id))
+        blob-store (:blob-store nimbus)
+        storm-conf (read-storm-conf conf storm-id blob-store)
+        topology (system-topology! storm-conf (read-storm-topology storm-id blob-store))
         num-executors (->> (all-components topology) (map-val num-start-executors))]
     (log-message "Activating " storm-name ": " storm-id)
     (.activate-storm! storm-cluster-state
@@ -935,17 +1040,15 @@
   ([nimbus storm-name storm-conf operation]
      (check-authorization! nimbus storm-name storm-conf operation (ReqContext/context))))
 
-(defn code-ids [conf]
-  (-> conf
-      master-stormdist-root
-      read-dir-contents
-      set
-      ))
+(defn code-ids [blob-store]
+  (let [to-id (reify KeyFilter
+                (filter [this key] (get-id-from-blob-key key)))]
+    (set (.filterAndListKeys blob-store to-id))))
 
-(defn cleanup-storm-ids [conf storm-cluster-state]
+(defn cleanup-storm-ids [conf storm-cluster-state blob-store]
   (let [heartbeat-ids (set (.heartbeat-storms storm-cluster-state))
         error-ids (set (.error-topologies storm-cluster-state))
-        code-ids (code-ids conf)
+        code-ids (code-ids blob-store)
         assigned-ids (set (.active-storms storm-cluster-state))]
     (set/difference (set/union heartbeat-ids error-ids code-ids) assigned-ids)
     ))
@@ -1006,22 +1109,35 @@
             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 nimbus-subject)
+    (if (instance? LocalFsBlobStore blob-store)
+      (.remove-blobstore-key! storm-cluster-state key))
+    (catch Exception e
+      (log-message "Exception" e))))
+
+(defn blob-rm-topology-keys [id blob-store storm-cluster-state]
+  (blob-rm-key blob-store (master-stormjar-key id) storm-cluster-state)
+  (blob-rm-key blob-store (master-stormconf-key id) storm-cluster-state)
+  (blob-rm-key blob-store (master-stormcode-key id) storm-cluster-state))
+
 (defn do-cleanup [nimbus]
   (if (is-leader nimbus :throw-exception false)
     (let [storm-cluster-state (:storm-cluster-state nimbus)
           conf (:conf nimbus)
-          submit-lock (:submit-lock nimbus)]
+          submit-lock (:submit-lock nimbus)
+          blob-store (:blob-store nimbus)]
       (let [to-cleanup-ids (locking submit-lock
-                             (cleanup-storm-ids conf storm-cluster-state))]
+                             (cleanup-storm-ids conf storm-cluster-state blob-store))]
         (when-not (empty? to-cleanup-ids)
           (doseq [id to-cleanup-ids]
             (log-message "Cleaning up " id)
-            (if (:code-distributor nimbus) (.cleanup (:code-distributor nimbus) id))
             (.teardown-heartbeats! storm-cluster-state id)
             (.teardown-topology-errors! storm-cluster-state id)
             (rmr (master-stormdist-root conf id))
-            (swap! (:heartbeats-cache nimbus) dissoc id))
-          )))
+            (blob-rm-topology-keys id blob-store storm-cluster-state)
+            (swap! (:heartbeats-cache nimbus) dissoc id)))))
     (log-message "not a leader, skipping cleanup")))
 
 (defn- file-older-than? [now seconds file]
@@ -1036,8 +1152,7 @@
       (if (.delete f)
         (log-message "Cleaning inbox ... deleted: " (.getName f))
         ;; This should never happen
-        (log-error "Cleaning inbox ... error deleting: " (.getName f))
-        ))))
+        (log-error "Cleaning inbox ... error deleting: " (.getName f))))))
 
 (defn clean-topology-history
   "Deletes topologies from history older than minutes."
@@ -1051,25 +1166,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))))))
+
+(defn setup-blobstore [nimbus]
+  "Sets up blobstore state for all current keys."
+  (let [storm-cluster-state (:storm-cluster-state nimbus)
+        blob-store (:blob-store nimbus)
+        local-set-of-keys (set (get-key-seq-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)
+        keys-to-delete (set/difference local-set-of-keys all-keys)
+        conf (:conf nimbus)
+        nimbus-host-port-info (:nimbus-host-port-info nimbus)]
+    (log-debug "Deleting keys not on the zookeeper" keys-to-delete)
+    (doseq [key keys-to-delete]
+      (.deleteBlob blob-store key nimbus-subject))
+    (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-version-for-key key nimbus-host-port-info conf)))))
 
 (defn- get-errors [storm-cluster-state storm-id component-id]
   (->> (.errors storm-cluster-state storm-id component-id)
@@ -1102,26 +1226,26 @@
     (catch Exception e
       (throw (AuthorizationException. (str "Invalid file path: " file-path))))))
 
-(defn try-read-storm-conf [conf storm-id]
+(defn try-read-storm-conf
+  [conf storm-id blob-store]
   (try-cause
-    (read-storm-conf conf storm-id)
-    (catch FileNotFoundException e
-       (throw (NotAliveException. (str storm-id))))
-  )
-)
+    (read-storm-conf-as-nimbus storm-id blob-store)
+    (catch KeyNotFoundException e
+      (throw (NotAliveException. (str storm-id))))))
 
-(defn try-read-storm-conf-from-name [conf storm-name nimbus]
+(defn try-read-storm-conf-from-name
+  [conf storm-name nimbus]
   (let [storm-cluster-state (:storm-cluster-state nimbus)
+        blob-store (:blob-store nimbus)
         id (get-storm-id storm-cluster-state storm-name)]
-   (try-read-storm-conf conf id)))
+    (try-read-storm-conf conf id blob-store)))
 
-(defn try-read-storm-topology [conf storm-id]
+(defn try-read-storm-topology
+  [storm-id blob-store]
   (try-cause
-    (read-storm-topology conf storm-id)
-    (catch FileNotFoundException e
-       (throw (NotAliveException. (str storm-id))))
-  )
-)
+    (read-storm-topology-as-nimbus storm-id blob-store)
+    (catch KeyNotFoundException e
+      (throw (NotAliveException. (str storm-id))))))
 
 (defn add-topology-to-history-log
   [storm-id nimbus topology-conf]
@@ -1166,6 +1290,7 @@
 (defn renew-credentials [nimbus]
   (if (is-leader nimbus :throw-exception false)
     (let [storm-cluster-state (:storm-cluster-state nimbus)
+          blob-store (:blob-store nimbus)
           renewers (:cred-renewers nimbus)
           update-lock (:cred-update-lock nimbus)
           assigned-ids (set (.active-storms storm-cluster-state))]
@@ -1173,7 +1298,7 @@
         (doseq [id assigned-ids]
           (locking update-lock
             (let [orig-creds (.credentials storm-cluster-state id nil)
-                  topology-conf (try-read-storm-conf (:conf nimbus) id)]
+                  topology-conf (try-read-storm-conf (:conf nimbus) id blob-store)]
               (if orig-creds
                 (let [new-creds (HashMap. orig-creds)]
                   (doseq [renewer renewers]
@@ -1210,22 +1335,40 @@
      (.set_reset_log_level_timeout_epoch log-config (coerce/to-long timeout))
      (.unset_reset_log_level_timeout_epoch log-config))))
 
+(defmethod blob-sync :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-set (set (get-key-seq-from-blob-store (:blob-store nimbus)))
+          zk-key-set (set (.blobstore storm-cluster-state (fn [] (blob-sync conf nimbus))))]
+      (log-debug "blob-sync " "blob-store-keys " blob-store-key-set "zookeeper-keys " zk-key-set)
+      (let [sync-blobs (doto
+                          (BlobSynchronizer. (:blob-store nimbus) conf)
+                          (.setNimbusInfo nimbus-host-port-info)
+                          (.setBlobStoreKeySet blob-store-key-set)
+                          (.setZookeeperKeySet zk-key-set))]
+        (.syncBlobs sync-blobs)))))
+
+(defmethod blob-sync :local [conf nimbus]
+  nil)
+
 (defserverfn service-handler [conf inimbus]
   (.prepare inimbus conf (master-inimbus-dir conf))
   (log-message "Starting Nimbus with conf " conf)
   (let [nimbus (nimbus-data conf inimbus)
+        blob-store (:blob-store nimbus)
         principal-to-local (AuthUtils/GetPrincipalToLocalPlugin conf)
         admin-users (or (.get conf NIMBUS-ADMINS) [])
         get-common-topo-info
           (fn [^String storm-id operation]
             (let [storm-cluster-state (:storm-cluster-state nimbus)
-                  topology-conf (try-read-storm-conf conf storm-id)
+                  topology-conf (try-read-storm-conf conf storm-id blob-store)
                   storm-name (topology-conf TOPOLOGY-NAME)
                   _ (check-authorization! nimbus
                                           storm-name
                                           topology-conf
                                           operation)
-                  topology (try-read-storm-topology conf storm-id)
+                  topology (try-read-storm-topology storm-id blob-store)
                   task->component (storm-task-info topology topology-conf)
                   base (.storm-base storm-cluster-state storm-id nil)
                   launch-time-secs (if base (:launch-time-secs base)
@@ -1264,10 +1407,11 @@
 
     (.addToLeaderLockQueue (:leader-elector nimbus))
     (cleanup-corrupt-topologies! nimbus)
-    (setup-code-distributor nimbus)
+    (when (instance? LocalFsBlobStore blob-store)
+      ;register call back for blob-store
+      (.blobstore (:storm-cluster-state nimbus) (fn [] (blob-sync conf nimbus)))
+      (setup-blobstore nimbus))
 
-    ;register call back for code-distributor
-    (.code-distributor (:storm-cluster-state nimbus) (fn [] (sync-code conf nimbus)))
     (when (is-leader nimbus :throw-exception false)
       (doseq [storm-id (.active-storms (:storm-cluster-state nimbus))]
         (transition! nimbus storm-id :startup)))
@@ -1278,31 +1422,27 @@
                           (when-not (conf NIMBUS-DO-NOT-REASSIGN)
                             (locking (:submit-lock nimbus)
                               (mk-assignments nimbus)))
-                          (do-cleanup nimbus)
-                          ))
+                          (do-cleanup nimbus)))
     ;; Schedule Nimbus inbox cleaner
     (schedule-recurring (:timer nimbus)
                         0
                         (conf NIMBUS-CLEANUP-INBOX-FREQ-SECS)
                         (fn []
-                          (clean-inbox (inbox nimbus) (conf NIMBUS-INBOX-JAR-EXPIRATION-SECS))
-                          ))
+                          (clean-inbox (inbox nimbus) (conf NIMBUS-INBOX-JAR-EXPIRATION-SECS))))
+    ;; 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 conf nimbus))))
     ;; Schedule topology history cleaner
     (when-let [interval (conf LOGVIEWER-CLEANUP-INTERVAL-SECS)]
       (schedule-recurring (:timer nimbus)
         0
         (conf LOGVIEWER-CLEANUP-INTERVAL-SECS)
         (fn []
-          (clean-topology-history (conf LOGVIEWER-CLEANUP-AGE-MINS) nimbus)
-          )))
-    ;;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)
-        ))
-
+          (clean-topology-history (conf LOGVIEWER-CLEANUP-AGE-MINS) nimbus))))
     (schedule-recurring (:timer nimbus)
                         0
                         (conf NIMBUS-CREDENTIAL-RENEW-FREQ-SECS)
@@ -1349,10 +1489,11 @@
                 principal (.principal req)
                 submitter-principal (if principal (.toString principal))
                 submitter-user (.toLocal principal-to-local principal)
+                system-user (System/getProperty "user.name")
                 topo-acl (distinct (remove nil? (conj (.get storm-conf-submitted TOPOLOGY-USERS) submitter-principal, submitter-user)))
                 storm-conf (-> storm-conf-submitted
                                (assoc TOPOLOGY-SUBMITTER-PRINCIPAL (if submitter-principal submitter-principal ""))
-                               (assoc TOPOLOGY-SUBMITTER-USER (if submitter-user submitter-user "")) ;Don't let the user set who we launch as
+                               (assoc TOPOLOGY-SUBMITTER-USER (if submitter-user submitter-user system-user)) ;Don't let the user set who we launch as
                                (assoc TOPOLOGY-USERS topo-acl)
                                (assoc STORM-ZOOKEEPER-SUPERACL (.get conf STORM-ZOOKEEPER-SUPERACL)))
                 storm-conf (if (Utils/isZkAuthenticationConfiguredStormServer conf)
@@ -1380,8 +1521,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)
+              (setup-storm-code nimbus conf storm-id uploadedJarLocation total-storm-conf topology)
               (wait-for-desired-code-replication nimbus total-storm-conf storm-id)
               (.setup-heartbeats! storm-cluster-state storm-id)
               (.setup-backpressure! storm-cluster-state storm-id)
@@ -1456,7 +1597,7 @@
         (mark! nimbus:num-debug-calls)
         (let [storm-cluster-state (:storm-cluster-state nimbus)
               storm-id (get-storm-id storm-cluster-state storm-name)
-              topology-conf (try-read-storm-conf conf storm-id)
+              topology-conf (try-read-storm-conf conf storm-id blob-store)
               ;; make sure samplingPct is within bounds.
               spct (Math/max (Math/min samplingPct 100.0) 0.0)
               ;; while disabling we retain the sampling pct.
@@ -1475,7 +1616,7 @@
       (^void setWorkerProfiler
         [this ^String id ^ProfileRequest profileRequest]
         (mark! nimbus:num-setWorkerProfiler-calls)
-        (let [topology-conf (try-read-storm-conf conf id)
+        (let [topology-conf (try-read-storm-conf conf id (:blob-store nimbus))
               storm-name (topology-conf TOPOLOGY-NAME)
               _ (check-authorization! nimbus storm-name topology-conf "setWorkerProfiler")
               storm-cluster-state (:storm-cluster-state nimbus)]
@@ -1506,7 +1647,7 @@
 
       (^void setLogConfig [this ^String id ^LogConfig log-config-msg]
         (mark! nimbus:num-setLogConfig-calls)
-        (let [topology-conf (try-read-storm-conf conf id)
+        (let [topology-conf (try-read-storm-conf conf id (:blob-store nimbus))
               storm-name (topology-conf TOPOLOGY-NAME)
               _ (check-authorization! nimbus storm-name topology-conf "setLogConfig")
               storm-cluster-state (:storm-cluster-state nimbus)
@@ -1534,7 +1675,7 @@
         (mark! nimbus:num-uploadNewCredentials-calls)
         (let [storm-cluster-state (:storm-cluster-state nimbus)
               storm-id (get-storm-id storm-cluster-state storm-name)
-              topology-conf (try-read-storm-conf conf storm-id)
+              topology-conf (try-read-storm-conf conf storm-id blob-store)
               creds (when credentials (.get_creds credentials))]
           (check-authorization! nimbus storm-name topology-conf "uploadNewCredentials")
           (locking (:cred-update-lock nimbus) (.set-credentials! storm-cluster-state storm-id creds topology-conf))))
@@ -1607,7 +1748,7 @@
 
       (^LogConfig getLogConfig [this ^String id]
         (mark! nimbus:num-getLogConfig-calls)
-        (let [topology-conf (try-read-storm-conf conf id)
+        (let [topology-conf (try-read-storm-conf conf id (:blob-store nimbus))
               storm-name (topology-conf TOPOLOGY-NAME)
               _ (check-authorization! nimbus storm-name topology-conf "getLogConfig")
              storm-cluster-state (:storm-cluster-state nimbus)
@@ -1616,24 +1757,24 @@
 
       (^String getTopologyConf [this ^String id]
         (mark! nimbus:num-getTopologyConf-calls)
-        (let [topology-conf (try-read-storm-conf conf id)
+        (let [topology-conf (try-read-storm-conf conf id (:blob-store nimbus))
               storm-name (topology-conf TOPOLOGY-NAME)]
               (check-authorization! nimbus storm-name topology-conf "getTopologyConf")
               (to-json topology-conf)))
 
       (^StormTopology getTopology [this ^String id]
         (mark! nimbus:num-getTopology-calls)
-        (let [topology-conf (try-read-storm-conf conf id)
+        (let [topology-conf (try-read-storm-conf conf id (:blob-store nimbus))
               storm-name (topology-conf TOPOLOGY-NAME)]
               (check-authorization! nimbus storm-name topology-conf "getTopology")
               (system-topology! topology-conf (try-read-storm-topology conf id))))
 
       (^StormTopology getUserTopology [this ^String id]
         (mark! nimbus:num-getUserTopology-calls)
-        (let [topology-conf (try-read-storm-conf conf id)
+        (let [topology-conf (try-read-storm-conf conf id (:blob-store nimbus))
               storm-name (topology-conf TOPOLOGY-NAME)]
               (check-authorization! nimbus storm-name topology-conf "getUserTopology")
-              (try-read-storm-topology topology-conf id)))
+              (try-read-storm-topology id blob-store)))
 
       (^ClusterSummary getClusterInfo [this]
         (mark! nimbus:num-getClusterInfo-calls)
@@ -1668,42 +1809,39 @@
                     (.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)
-                                                topo-summ (TopologySummary. id
-                                                            (:storm-name base)
-                                                            (->> (:executor->node+port assignment)
-                                                                 keys
-                                                                 (mapcat executor-id->tasks)
-                                                                 count) 
-                                                            (->> (:executor->node+port assignment)
-                                                                 keys
-                                                                 count)                                                            
-                                                            (->> (:executor->node+port assignment)
-                                                                 vals
-                                                                 set
-                                                                 count)
-                                                            (time-delta (:launch-time-secs base))
-                                                            (extract-status-str base))]
-                                               (when-let [owner (:owner base)] (.set_owner topo-summ owner))
-                                               (when-let [sched-status (.get @(:id->sched-status nimbus) id)] (.set_sched_status topo-summ sched-status))
-                                               (when-let [resources (.get @(:id->resources nimbus) id)]
-                                                 (.set_requested_memonheap topo-summ (get resources 0))
-                                                 (.set_requested_memoffheap topo-summ (get resources 1))
-                                                 (.set_requested_cpu topo-summ (get resources 2))
-                                                 (.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))
-                                               topo-summ
-                                          ))
+                                   (let [assignment (.assignment-info storm-cluster-state id nil)
+                                         topo-summ (TopologySummary. id
+                                                     (:storm-name base)
+                                                     (->> (:executor->node+port assignment)
+                                                       keys
+                                                       (mapcat executor-id->tasks)
+                                                       count)
+                                                     (->> (:executor->node+port assignment)
+                                                       keys
+                                                       count)
+                                                     (->> (:executor->node+port assignment)
+                                                       vals
+                                                       set
+                                                       count)
+                                                     (time-delta (:launch-time-secs base))
+                                                     (extract-status-str base))]
+                                     (when-let [owner (:owner base)] (.set_owner topo-summ owner))
+                                     (when-let [sched-status (.get @(:id->sched-status nimbus) id)] (.set_sched_status topo-summ sched-status))
+                                     (when-let [resources (.get @(:id->resources nimbus) id)]
+                                       (.set_requested_memonheap topo-summ (get resources 0))
+                                       (.set_requested_memoffheap topo-summ (get resources 1))
+                                       (.set_requested_cpu topo-summ (get resources 2))
+                                       (.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 (get-blob-replication-count (master-stormcode-key id) nimbus))
+                                     topo-summ))
               ret (ClusterSummary. supervisor-summaries
                                    topology-summaries
                                    nimbuses)
               _ (.set_nimbus_uptime_secs ret nimbus-uptime)]
               ret))
-      
+
       (^TopologyInfo getTopologyInfoWithOpts [this ^String storm-id ^GetInfoOptions options]
         (mark! nimbus:num-getTopologyInfoWithOpts-calls)
         (let [{:keys [storm-name
@@ -1763,10 +1901,8 @@
               (.set_assigned_cpu topo-info (get resources 5)))
             (when-let [component->debug (:component->debug base)]
               (.set_component_debug topo-info (map-val converter/thriftify-debugoptions component->debug)))
-            (.set_replication_count topo-info (if (:code-distributor nimbus) (.getReplicationCount (:code-distributor nimbus) storm-id) 1))
-
-            topo-info
-          ))
+            (.set_replication_count topo-info (get-blob-replication-count (master-stormcode-key storm-id) nimbus))
+          topo-info))
 
       (^TopologyInfo getTopologyInfo [this ^String topology-id]
         (mark! nimbus:num-getTopologyInfo-calls)
@@ -1774,6 +1910,157 @@
                                   topology-id
                                   (doto (GetInfoOptions.) (.set_num_err_choice NumErrorsChoice/ALL))))
 
+      (^String beginCreateBlob [this
+                                ^String blob-key
+                                ^SettableBlobMeta blob-meta]
+        (let [session-id (uuid)]
+          (.put (:blob-uploaders nimbus)
+            session-id
+            (.createBlob (:blob-store nimbus) blob-key blob-meta user-subject))
+          (log-message "Created blob for " blob-key
+            " with session id " session-id)
+          (str session-id)))
+
+      (^String beginUpdateBlob [this ^String blob-key]
+        (let [^AtomicOutputStream os (.updateBlob (:blob-store nimbus)
+                                       blob-key user-subject)]
+          (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 (.getBlobMeta (:blob-store nimbus)
+                                      blob-key user-subject)]
+          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 (.getBlob (:blob-store nimbus)
+                                        blob-key user-subject)]
+          (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 (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)
+                                  (.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)))
+
       (^TopologyPageInfo getTopologyPageInfo
         [this ^String topo-id ^String window ^boolean include-sys?]
         (mark! nimbus:num-getTopologyPageInfo-calls)
@@ -1807,9 +2094,8 @@
             (.set_status (extract-status-str (:base info)))
             (.set_uptime_secs (time-delta (:launch-time-secs info)))
             (.set_topology_conf (to-json (try-read-storm-conf conf
-                                                              topo-id)))
-            (.set_replication_count
-              (.getReplicationCount (:code-distributor nimbus) topo-id)))
+                                                              topo-id (:blob-store nimbus))))
+            (.set_replication_count (get-blob-replication-count (master-stormcode-key topo-id) nimbus)))
           (when-let [debug-options
                      (get-in info [:base :component->debug topo-id])]
             (.set_debug_options
@@ -1889,60 +2175,14 @@
         (.disconnect (:storm-cluster-state nimbus))
         (.cleanup (:downloaders nimbus))
         (.cleanup (:uploaders nimbus))
+        (.shutdown (:blob-store nimbus))
         (.close (:leader-elector nimbus))
-        (if (:code-distributor nimbus) (.close (:code-distributor nimbus) (:conf nimbus)))
         (when (:nimbus-topology-action-notifier nimbus) (.cleanup (:nimbus-topology-action-notifier nimbus)))
-        (log-message "Shut down master")
-        )
+        (log-message "Shut down master"))
       DaemonCommon
       (waiting? [this]
         (timer-waiting? (:timer nimbus))))))
 
-(defmethod mk-code-distributor :distributed [conf]
-  (let [code-distributor (new-instance (conf STORM-CODE-DISTRIBUTOR-CLASS))]
-    (.prepare code-distributor conf)
-    code-distributor))
-
-(defmethod mk-code-distributor :local [conf]
-  nil)
-
-(defn download-code [conf nimbus storm-id host port]
-  (let [tmp-root (str (master-tmp-dir conf) file-path-separator (uuid))
-        storm-cluster-state (:storm-cluster-state nimbus)
-        storm-root (master-stormdist-root conf storm-id)
-        remote-meta-file-path (master-storm-metafile-path storm-root)
-        local-meta-file-path (master-storm-metafile-path tmp-root)]
-    (FileUtils/forceMkdir (File. tmp-root))
-    (Utils/downloadFromHost conf remote-meta-file-path local-meta-file-path host port)
-    (if (:code-distributor nimbus)
-      (.download (:code-distributor nimbus) storm-id (File. local-meta-file-path)))
-    (if (.exists (File. storm-root)) (FileUtils/forceDelete (File. storm-root)))
-    (FileUtils/moveDirectory (File. tmp-root) (File. storm-root))
-    (.setup-code-distributor! storm-cluster-state storm-id (:nimbus-host-port-info nimbus))))
-
-(defmethod sync-code :distributed [conf nimbus]
-  (let [storm-cluster-state (:storm-cluster-state nimbus)
-        active-topologies (set (.code-distributor storm-cluster-state (fn [] (sync-code conf nimbus))))
-        missing-topologies (set/difference active-topologies (set (code-ids (:conf nimbus))))]
-    (if (not (empty? missing-topologies))
-      (do
-        (.removeFromLeaderLockQueue (:leader-elector nimbus))
-        (doseq [missing missing-topologies]
-          (log-message "missing topology " missing " has state on zookeeper but doesn't have a local dir on this host.")
-          (let [nimbuses-with-missing (.code-distributor-info storm-cluster-state missing)]
-            (log-message "trying to download missing topology code from " (clojure.string/join "," nimbuses-with-missing))
-            (doseq [nimbus-host-port nimbuses-with-missing]
-              (when-not (contains? (code-ids (:conf nimbus)) missing)
-                (try
-                  (download-code conf nimbus missing (.getHost nimbus-host-port) (.getPort nimbus-host-port))
-                  (catch Exception e (log-error e "Exception while trying to syn-code for missing topology" missing)))))))))
-
-    (if (empty? (set/difference active-topologies (set (code-ids (:conf nimbus)))))
-      (.addToLeaderLockQueue (:leader-elector nimbus)))))
-
-(defmethod sync-code :local [conf nimbus]
-  nil)
-
 (defn launch-server! [conf nimbus]
   (validate-distributed-mode! conf)
   (let [service-handler (service-handler conf nimbus)


[04/17] storm git commit: Blobstore API STORM- 876

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/localizer/Localizer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/localizer/Localizer.java b/storm-core/src/jvm/backtype/storm/localizer/Localizer.java
new file mode 100644
index 0000000..ef5684f
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/localizer/Localizer.java
@@ -0,0 +1,695 @@
+/**
+ * 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.localizer;
+
+import backtype.storm.Config;
+import backtype.storm.blobstore.ClientBlobStore;
+import backtype.storm.blobstore.InputStreamWithMeta;
+import backtype.storm.generated.AuthorizationException;
+import backtype.storm.generated.KeyNotFoundException;
+import backtype.storm.utils.ShellUtils.ExitCodeException;
+import backtype.storm.utils.ShellUtils.ShellCommandExecutor;
+import backtype.storm.utils.Utils;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.commons.io.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FileWriter;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import static java.nio.file.StandardCopyOption.ATOMIC_MOVE;
+
+/**
+ * Class to download and manage files from the blobstore.  It uses an LRU cache
+ * to determine which files to keep so they can be reused and which files to delete.
+ */
+public class Localizer {
+  public static final Logger LOG = LoggerFactory.getLogger(Localizer.class);
+
+  private Map _conf;
+  private int _threadPoolSize;
+  // thread pool for initial download
+  private ExecutorService _execService;
+  // thread pool for updates
+  private ExecutorService _updateExecService;
+  private int _blobDownloadRetries;
+
+  // track resources - user to resourceSet
+  private final ConcurrentMap<String, LocalizedResourceSet> _userRsrc = new
+      ConcurrentHashMap<String, LocalizedResourceSet>();
+
+  private String _localBaseDir;
+  public static final String USERCACHE = "usercache";
+  public static final String FILECACHE = "filecache";
+
+  // sub directories to store either files or uncompressed archives respectively
+  public static final String FILESDIR = "files";
+  public static final String ARCHIVESDIR = "archives";
+
+  private static final String TO_UNCOMPRESS = "_tmp_";
+
+  // cleanup
+  private long _cacheTargetSize;
+  private long _cacheCleanupPeriod;
+  private ScheduledExecutorService _cacheCleanupService;
+
+  public Localizer(Map conf, String baseDir) {
+    _conf = conf;
+    _localBaseDir = baseDir;
+    // default cache size 10GB, converted to Bytes
+    _cacheTargetSize = Utils.getInt(_conf.get(Config.SUPERVISOR_LOCALIZER_CACHE_TARGET_SIZE_MB),
+            10 * 1024).longValue() << 20;
+    // default 10 minutes.
+    _cacheCleanupPeriod = Utils.getInt(_conf.get(
+            Config.SUPERVISOR_LOCALIZER_CACHE_CLEANUP_INTERVAL_MS), 10 * 60 * 1000).longValue();
+
+    // if we needed we could make config for update thread pool size
+    _threadPoolSize = Utils.getInt(_conf.get(Config.SUPERVISOR_BLOBSTORE_DOWNLOAD_THREAD_COUNT), 5);
+    _blobDownloadRetries = Utils.getInt(_conf.get(
+            Config.SUPERVISOR_BLOBSTORE_DOWNLOAD_MAX_RETRIES), 3);
+
+    _execService = Executors.newFixedThreadPool(_threadPoolSize);
+    _updateExecService = Executors.newFixedThreadPool(_threadPoolSize);
+    reconstructLocalizedResources();
+  }
+
+  // For testing, it allows setting size in bytes
+  protected void setTargetCacheSize(long size) {
+    _cacheTargetSize = size;
+  }
+
+  // For testing, be careful as it doesn't clone
+  ConcurrentMap<String, LocalizedResourceSet> getUserResources() {
+    return _userRsrc;
+  }
+
+  public void startCleaner() {
+    _cacheCleanupService = new ScheduledThreadPoolExecutor(1,
+        new ThreadFactoryBuilder()
+            .setNameFormat("Localizer Cache Cleanup")
+            .build());
+
+    _cacheCleanupService.scheduleWithFixedDelay(new Runnable() {
+          @Override
+          public void run() {
+            handleCacheCleanup();
+          }
+        }, _cacheCleanupPeriod, _cacheCleanupPeriod, TimeUnit.MILLISECONDS);
+  }
+
+  public void shutdown() {
+    if (_cacheCleanupService != null) {
+      _cacheCleanupService.shutdown();
+    }
+    if (_execService != null) {
+      _execService.shutdown();
+    }
+    if (_updateExecService != null) {
+      _updateExecService.shutdown();
+    }
+  }
+
+  // baseDir/supervisor/usercache/
+  protected File getUserCacheDir() {
+    return new File(_localBaseDir, USERCACHE);
+  }
+
+  // baseDir/supervisor/usercache/user1/
+  protected File getLocalUserDir(String userName) {
+    return new File(getUserCacheDir(), userName);
+  }
+
+  // baseDir/supervisor/usercache/user1/filecache
+  public File getLocalUserFileCacheDir(String userName) {
+    return new File(getLocalUserDir(userName), FILECACHE);
+  }
+
+  // baseDir/supervisor/usercache/user1/filecache/files
+  protected File getCacheDirForFiles(File dir) {
+    return new File(dir, FILESDIR);
+  }
+
+  // get the directory to put uncompressed archives in
+  // baseDir/supervisor/usercache/user1/filecache/archives
+  protected File getCacheDirForArchives(File dir) {
+    return new File(dir, ARCHIVESDIR);
+  }
+
+  protected void addLocalizedResourceInDir(String dir, LocalizedResourceSet lrsrcSet,
+      boolean uncompress) {
+    File[] lrsrcs = readCurrentBlobs(dir);
+
+    if (lrsrcs != null) {
+      for (File rsrc : lrsrcs) {
+        LOG.info("add localized in dir found: " + rsrc);
+        /// strip off .suffix
+        String path = rsrc.getPath();
+        int p = path.lastIndexOf('.');
+        if (p > 0) {
+          path = path.substring(0, p);
+        }
+        LOG.debug("local file is: {} path is: {}", rsrc.getPath(), path);
+        LocalizedResource lrsrc = new LocalizedResource(new File(path).getName(), path,
+            uncompress);
+        lrsrcSet.addResource(lrsrc.getKey(), lrsrc, uncompress);
+      }
+    }
+  }
+
+  protected File[] readDirContents(String location) {
+    File dir = new File(location);
+    File[] files = null;
+    if (dir.exists()) {
+      files = dir.listFiles();
+    }
+    return files;
+  }
+
+  // Looks for files in the directory with .current suffix
+  protected File[] readCurrentBlobs(String location) {
+    File dir = new File(location);
+    File[] files = null;
+    if (dir.exists()) {
+      files = dir.listFiles(new FilenameFilter() {
+        @Override
+        public boolean accept(File dir, String name) {
+          return name.toLowerCase().endsWith(Utils.DEFAULT_CURRENT_BLOB_SUFFIX);
+        }
+      });
+    }
+    return files;
+  }
+
+  // Check to see if there are any existing files already localized.
+  protected void reconstructLocalizedResources() {
+    try {
+      LOG.info("Reconstruct localized resource: " + getUserCacheDir().getPath());
+      File[] users = readDirContents(getUserCacheDir().getPath());
+
+      if (users != null) {
+        for (File userDir : users) {
+          String user = userDir.getName();
+          LOG.debug("looking in: {} for user: {}", userDir.getPath(), user);
+          LocalizedResourceSet newSet = new LocalizedResourceSet(user);
+          LocalizedResourceSet lrsrcSet = _userRsrc.putIfAbsent(user, newSet);
+          if (lrsrcSet == null) {
+            lrsrcSet = newSet;
+          }
+          addLocalizedResourceInDir(getCacheDirForFiles(getLocalUserFileCacheDir(user)).getPath(),
+              lrsrcSet, false);
+          addLocalizedResourceInDir(
+              getCacheDirForArchives(getLocalUserFileCacheDir(user)).getPath(),
+              lrsrcSet, true);
+        }
+      } else {
+        LOG.warn("No left over resources found for any user during reconstructing of local resources at: {}", getUserCacheDir().getPath());
+      }
+    } catch (Exception e) {
+      LOG.error("ERROR reconstructing localized resources", e);
+    }
+  }
+
+  // ignores invalid user/topo/key
+  public synchronized void removeBlobReference(String key, String user, String topo,
+      boolean uncompress) throws AuthorizationException, KeyNotFoundException {
+    LocalizedResourceSet lrsrcSet = _userRsrc.get(user);
+    if (lrsrcSet != null) {
+      LocalizedResource lrsrc = lrsrcSet.get(key, uncompress);
+      if (lrsrc != null) {
+        LOG.debug("removing blob reference to: {} for topo: {}", key, topo);
+        lrsrc.removeReference(topo);
+      } else {
+        LOG.warn("trying to remove non-existent blob, key: " + key + " for user: " + user +
+            " topo: " + topo);
+      }
+    } else {
+      LOG.warn("trying to remove blob for non-existent resource set for user: " + user + " key: "
+          + key + " topo: " + topo);
+    }
+  }
+
+  public synchronized void addReferences(List<LocalResource> localresource, String user,
+       String topo) {
+    LocalizedResourceSet lrsrcSet = _userRsrc.get(user);
+    if (lrsrcSet != null) {
+      for (LocalResource blob : localresource) {
+        LocalizedResource lrsrc = lrsrcSet.get(blob.getBlobName(), blob.shouldUncompress());
+        if (lrsrc != null) {
+          lrsrc.addReference(topo);
+          LOG.debug("added reference for topo: {} key: {}", topo, blob);
+        } else {
+          LOG.warn("trying to add reference to non-existent blob, key: " + blob + " topo: " + topo);
+        }
+      }
+    } else {
+      LOG.warn("trying to add reference to non-existent local resource set, " +
+          "user: " + user + " topo: " + topo);
+    }
+  }
+
+  /**
+   * This function either returns the blob in the existing cache or if it doesn't exist in the
+   * cache, it will download the blob and will block until the download is complete.
+   */
+  public LocalizedResource getBlob(LocalResource localResource, String user, String topo,
+       File userFileDir) throws AuthorizationException, KeyNotFoundException, IOException {
+    ArrayList<LocalResource> arr = new ArrayList<LocalResource>();
+    arr.add(localResource);
+    List<LocalizedResource> results = getBlobs(arr, user, topo, userFileDir);
+    if (results.isEmpty() || results.size() != 1) {
+      throw new IOException("Unknown error getting blob: " + localResource + ", for user: " + user +
+          ", topo: " + topo);
+    }
+    return results.get(0);
+  }
+
+  protected boolean isLocalizedResourceDownloaded(LocalizedResource lrsrc) {
+    File rsrcFileCurrent = new File(lrsrc.getCurrentSymlinkPath());
+    File rsrcFileWithVersion = new File(lrsrc.getFilePathWithVersion());
+    File versionFile = new File(lrsrc.getVersionFilePath());
+    return (rsrcFileWithVersion.exists() && rsrcFileCurrent.exists() && versionFile.exists());
+  }
+
+  protected boolean isLocalizedResourceUpToDate(LocalizedResource lrsrc,
+      ClientBlobStore blobstore) throws AuthorizationException, KeyNotFoundException {
+    String localFile = lrsrc.getFilePath();
+    long nimbusBlobVersion = Utils.nimbusVersionOfBlob(lrsrc.getKey(), blobstore);
+    long currentBlobVersion = Utils.localVersionOfBlob(localFile);
+    return (nimbusBlobVersion == currentBlobVersion);
+  }
+
+  protected ClientBlobStore getClientBlobStore() {
+    return Utils.getClientBlobStoreForSupervisor(_conf);
+  }
+
+  /**
+   * This function updates blobs on the supervisor. It uses a separate thread pool and runs
+   * asynchronously of the download and delete.
+   */
+  public List<LocalizedResource> updateBlobs(List<LocalResource> localResources,
+       String user) throws AuthorizationException, KeyNotFoundException, IOException {
+    LocalizedResourceSet lrsrcSet = _userRsrc.get(user);
+    ArrayList<LocalizedResource> results = new ArrayList<>();
+    ArrayList<Callable<LocalizedResource>> updates = new ArrayList<>();
+
+    if (lrsrcSet == null) {
+      // resource set must have been removed
+      return results;
+    }
+    ClientBlobStore blobstore = null;
+    try {
+      blobstore = getClientBlobStore();
+      for (LocalResource localResource: localResources) {
+        String key = localResource.getBlobName();
+        LocalizedResource lrsrc = lrsrcSet.get(key, localResource.shouldUncompress());
+        if (lrsrc == null) {
+          LOG.warn("blob requested for update doesn't exist: {}", key);
+          continue;
+        } else {
+          // update it if either the version isn't the latest or if any local blob files are missing
+          if (!isLocalizedResourceUpToDate(lrsrc, blobstore) ||
+              !isLocalizedResourceDownloaded(lrsrc)) {
+            LOG.debug("updating blob: {}", key);
+            updates.add(new DownloadBlob(this, _conf, key, new File(lrsrc.getFilePath()), user,
+                lrsrc.isUncompressed(), true));
+          }
+        }
+      }
+    } finally {
+      if(blobstore != null) {
+        blobstore.shutdown();
+      }
+    }
+    try {
+      List<Future<LocalizedResource>> futures = _updateExecService.invokeAll(updates);
+      for (Future<LocalizedResource> futureRsrc : futures) {
+        try {
+          LocalizedResource lrsrc = futureRsrc.get();
+          // put the resource just in case it was removed at same time by the cleaner
+          LocalizedResourceSet newSet = new LocalizedResourceSet(user);
+          LocalizedResourceSet newlrsrcSet = _userRsrc.putIfAbsent(user, newSet);
+          if (newlrsrcSet == null) {
+            newlrsrcSet = newSet;
+          }
+          newlrsrcSet.updateResource(lrsrc.getKey(), lrsrc, lrsrc.isUncompressed());
+          results.add(lrsrc);
+        }
+        catch (ExecutionException e) {
+          LOG.error("Error updating blob: ", e);
+          if (e.getCause() instanceof AuthorizationException) {
+            throw (AuthorizationException)e.getCause();
+          }
+          if (e.getCause() instanceof KeyNotFoundException) {
+            throw (KeyNotFoundException)e.getCause();
+          }
+        }
+      }
+    } catch (RejectedExecutionException re) {
+      LOG.error("Error updating blobs : ", re);
+    } catch (InterruptedException ie) {
+      throw new IOException("Interrupted Exception", ie);
+    }
+    return results;
+  }
+
+  /**
+   * This function either returns the blobs in the existing cache or if they don't exist in the
+   * cache, it downloads them in parallel (up to SUPERVISOR_BLOBSTORE_DOWNLOAD_THREAD_COUNT)
+   * and will block until all of them have been downloaded
+   */
+  public synchronized List<LocalizedResource> getBlobs(List<LocalResource> localResources,
+      String user, String topo, File userFileDir)
+      throws AuthorizationException, KeyNotFoundException, IOException {
+
+    LocalizedResourceSet newSet = new LocalizedResourceSet(user);
+    LocalizedResourceSet lrsrcSet = _userRsrc.putIfAbsent(user, newSet);
+    if (lrsrcSet == null) {
+      lrsrcSet = newSet;
+    }
+    ArrayList<LocalizedResource> results = new ArrayList<>();
+    ArrayList<Callable<LocalizedResource>> downloads = new ArrayList<>();
+
+    ClientBlobStore blobstore = null;
+    try {
+      blobstore = getClientBlobStore();
+      for (LocalResource localResource: localResources) {
+        String key = localResource.getBlobName();
+        boolean uncompress = localResource.shouldUncompress();
+        LocalizedResource lrsrc = lrsrcSet.get(key, localResource.shouldUncompress());
+        boolean isUpdate = false;
+        if ((lrsrc != null) && (lrsrc.isUncompressed() == localResource.shouldUncompress()) &&
+            (isLocalizedResourceDownloaded(lrsrc))) {
+          if (isLocalizedResourceUpToDate(lrsrc, blobstore)) {
+            LOG.debug("blob already exists: {}", key);
+            lrsrc.addReference(topo);
+            results.add(lrsrc);
+            continue;
+          }
+          LOG.debug("blob exists but isn't up to date: {}", key);
+          isUpdate = true;
+        }
+
+        // go off to blobstore and get it
+        // assume dir passed in exists and has correct permission
+        LOG.debug("fetching blob: {}", key);
+        File downloadDir = getCacheDirForFiles(userFileDir);
+        File localFile = new File(downloadDir, key);
+        if (uncompress) {
+          // for compressed file, download to archives dir
+          downloadDir = getCacheDirForArchives(userFileDir);
+          localFile = new File(downloadDir, key);
+        }
+        downloadDir.mkdir();
+        downloads.add(new DownloadBlob(this, _conf, key, localFile, user, uncompress,
+            isUpdate));
+      }
+    } finally {
+      if(blobstore !=null) {
+        blobstore.shutdown();
+      }
+    }
+    try {
+      List<Future<LocalizedResource>> futures = _execService.invokeAll(downloads);
+      for (Future<LocalizedResource> futureRsrc: futures) {
+        LocalizedResource lrsrc = futureRsrc.get();
+        lrsrc.addReference(topo);
+        lrsrcSet.addResource(lrsrc.getKey(), lrsrc, lrsrc.isUncompressed());
+        results.add(lrsrc);
+      }
+    } catch (ExecutionException e) {
+      if (e.getCause() instanceof AuthorizationException)
+        throw (AuthorizationException)e.getCause();
+      else if (e.getCause() instanceof KeyNotFoundException) {
+        throw (KeyNotFoundException)e.getCause();
+      } else {
+        throw new IOException("Error getting blobs", e);
+      }
+    } catch (RejectedExecutionException re) {
+      throw new IOException("RejectedExecutionException: ", re);
+    } catch (InterruptedException ie) {
+      throw new IOException("Interrupted Exception", ie);
+    }
+    return results;
+  }
+
+  static class DownloadBlob implements Callable<LocalizedResource> {
+
+    private Localizer _localizer;
+    private Map _conf;
+    private String _key;
+    private File _localFile;
+    private String _user;
+    private boolean _uncompress;
+    private boolean _isUpdate;
+
+    public DownloadBlob(Localizer localizer, Map conf, String key, File localFile,
+        String user, boolean uncompress, boolean update) {
+      _localizer = localizer;
+      _conf = conf;
+      _key = key;
+      _localFile = localFile;
+      _user = user;
+      _uncompress = uncompress;
+      _isUpdate = update;
+    }
+
+    @Override
+    public LocalizedResource call()
+        throws AuthorizationException, KeyNotFoundException, IOException  {
+      return _localizer.downloadBlob(_conf, _key, _localFile, _user, _uncompress,
+        _isUpdate);
+    }
+  }
+
+  private LocalizedResource downloadBlob(Map conf, String key, File localFile,
+      String user, boolean uncompress, boolean isUpdate)
+      throws AuthorizationException, KeyNotFoundException, IOException {
+    ClientBlobStore blobstore = null;
+    try {
+      blobstore = getClientBlobStore();
+      long nimbusBlobVersion = Utils.nimbusVersionOfBlob(key, blobstore);
+      long oldVersion = Utils.localVersionOfBlob(localFile.toString());
+      FileOutputStream out = null;
+      PrintWriter writer = null;
+      int numTries = 0;
+      String localizedPath = localFile.toString();
+      String localFileWithVersion = Utils.constructBlobWithVersionFileName(localFile.toString(),
+              nimbusBlobVersion);
+      String localVersionFile = Utils.constructVersionFileName(localFile.toString());
+      String downloadFile = localFileWithVersion;
+      if (uncompress) {
+        // we need to download to temp file and then unpack into the one requested
+        downloadFile = new File(localFile.getParent(), TO_UNCOMPRESS + localFile.getName()).toString();
+      }
+      while (numTries < _blobDownloadRetries) {
+        out = new FileOutputStream(downloadFile);
+        numTries++;
+        try {
+          if (!Utils.canUserReadBlob(blobstore.getBlobMeta(key), user)) {
+            throw new AuthorizationException(user + " does not have READ access to " + key);
+          }
+          InputStreamWithMeta in = blobstore.getBlob(key);
+          byte[] buffer = new byte[1024];
+          int len;
+          while ((len = in.read(buffer)) >= 0) {
+            out.write(buffer, 0, len);
+          }
+          out.close();
+          in.close();
+          if (uncompress) {
+            Utils.unpack(new File(downloadFile), new File(localFileWithVersion));
+            LOG.debug("uncompressed " + downloadFile + " to: " + localFileWithVersion);
+          }
+
+          // Next write the version.
+          LOG.info("Blob: " + key + " updated with new Nimbus-provided version: " +
+              nimbusBlobVersion + " local version was: " + oldVersion);
+          // The false parameter ensures overwriting the version file, not appending
+          writer = new PrintWriter(
+              new BufferedWriter(new FileWriter(localVersionFile, false)));
+          writer.println(nimbusBlobVersion);
+          writer.close();
+
+          try {
+            setBlobPermissions(conf, user, localFileWithVersion);
+            setBlobPermissions(conf, user, localVersionFile);
+
+            // Update the key.current symlink. First create tmp symlink and do
+            // move of tmp to current so that the operation is atomic.
+            String tmp_uuid_local = java.util.UUID.randomUUID().toString();
+            LOG.debug("Creating a symlink @" + localFile + "." + tmp_uuid_local + " , " +
+                "linking to: " + localFile + "." + nimbusBlobVersion);
+            File uuid_symlink = new File(localFile + "." + tmp_uuid_local);
+
+            Files.createSymbolicLink(uuid_symlink.toPath(),
+                Paths.get(Utils.constructBlobWithVersionFileName(localFile.toString(),
+                        nimbusBlobVersion)));
+            File current_symlink = new File(Utils.constructBlobCurrentSymlinkName(
+                    localFile.toString()));
+            Files.move(uuid_symlink.toPath(), current_symlink.toPath(), ATOMIC_MOVE);
+          } catch (IOException e) {
+            // if we fail after writing the version file but before we move current link we need to
+            // restore the old version to the file
+            try {
+              PrintWriter restoreWriter = new PrintWriter(
+                  new BufferedWriter(new FileWriter(localVersionFile, false)));
+              restoreWriter.println(oldVersion);
+              restoreWriter.close();
+            } catch (IOException ignore) {}
+            throw e;
+          }
+
+          String oldBlobFile = localFile + "." + oldVersion;
+          try {
+            // Remove the old version. Note that if a number of processes have that file open,
+            // the OS will keep the old blob file around until they all close the handle and only
+            // then deletes it. No new process will open the old blob, since the users will open the
+            // blob through the "blob.current" symlink, which always points to the latest version of
+            // a blob. Remove the old version after the current symlink is updated as to not affect
+            // anyone trying to read it.
+            if ((oldVersion != -1) && (oldVersion != nimbusBlobVersion)) {
+              LOG.info("Removing an old blob file:" + oldBlobFile);
+              Files.delete(Paths.get(oldBlobFile));
+            }
+          } catch (IOException e) {
+            // At this point we have downloaded everything and moved symlinks.  If the remove of
+            // old fails just log an error
+            LOG.error("Exception removing old blob version: " + oldBlobFile);
+          }
+
+          break;
+        } catch (AuthorizationException ae) {
+          // we consider this non-retriable exceptions
+          if (out != null) {
+            out.close();
+          }
+          new File(downloadFile).delete();
+          throw ae;
+        } catch (IOException | KeyNotFoundException e) {
+          if (out != null) {
+            out.close();
+          }
+          if (writer != null) {
+            writer.close();
+          }
+          new File(downloadFile).delete();
+          if (uncompress) {
+            try {
+              FileUtils.deleteDirectory(new File(localFileWithVersion));
+            } catch (IOException ignore) {}
+          }
+          if (!isUpdate) {
+            // don't want to remove existing version file if its an update
+            new File(localVersionFile).delete();
+          }
+
+          if (numTries < _blobDownloadRetries) {
+            LOG.error("Failed to download blob, retrying", e);
+          } else {
+            throw e;
+          }
+        }
+      }
+      return new LocalizedResource(key, localizedPath, uncompress);
+    } finally {
+      if(blobstore != null) {
+        blobstore.shutdown();
+      }
+    }
+  }
+
+  public void setBlobPermissions(Map conf, String user, String path)
+      throws IOException {
+
+    if (!Utils.getBoolean(conf.get(Config.SUPERVISOR_RUN_WORKER_AS_USER), false)) {
+      return;
+    }
+    String wlCommand = Utils.getString(conf.get(Config.SUPERVISOR_WORKER_LAUNCHER), "");
+    if (wlCommand.isEmpty()) {
+      String stormHome = System.getProperty("storm.home");
+      wlCommand = stormHome + "/bin/worker-launcher";
+    }
+    List<String> command = new ArrayList<String>(Arrays.asList(wlCommand, user, "blob", path));
+
+    String[] commandArray = command.toArray(new String[command.size()]);
+    ShellCommandExecutor shExec = new ShellCommandExecutor(commandArray);
+    LOG.info("Setting blob permissions, command: {}", Arrays.toString(commandArray));
+
+    try {
+      shExec.execute();
+      LOG.debug("output: {}", shExec.getOutput());
+    } catch (ExitCodeException e) {
+      int exitCode = shExec.getExitCode();
+      LOG.warn("Exit code from worker-launcher is : " + exitCode, e);
+      LOG.debug("output: {}", shExec.getOutput());
+      throw new IOException("Setting blob permissions failed" +
+          " (exitCode=" + exitCode + ") with output: " + shExec.getOutput(), e);
+    }
+  }
+
+
+  public synchronized void handleCacheCleanup() {
+    LocalizedResourceRetentionSet toClean = new LocalizedResourceRetentionSet(_cacheTargetSize);
+    // need one large set of all and then clean via LRU
+    for (LocalizedResourceSet t : _userRsrc.values()) {
+      toClean.addResources(t);
+      LOG.debug("Resources to be cleaned after adding {} : {}", t.getUser(), toClean);
+    }
+    toClean.cleanup();
+    LOG.debug("Resource cleanup: {}", toClean);
+    for (LocalizedResourceSet t : _userRsrc.values()) {
+      if (t.getSize() == 0) {
+        String user = t.getUser();
+
+        LOG.debug("removing empty set: {}", user);
+        File userFileCacheDir = getLocalUserFileCacheDir(user);
+        getCacheDirForFiles(userFileCacheDir).delete();
+        getCacheDirForArchives(userFileCacheDir).delete();
+        getLocalUserFileCacheDir(user).delete();
+        boolean dirsRemoved = getLocalUserDir(user).delete();
+        // to catch race with update thread
+        if (dirsRemoved) {
+          _userRsrc.remove(user);
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/security/auth/NimbusPrincipal.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/NimbusPrincipal.java b/storm-core/src/jvm/backtype/storm/security/auth/NimbusPrincipal.java
new file mode 100644
index 0000000..c07ae84
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/security/auth/NimbusPrincipal.java
@@ -0,0 +1,29 @@
+/**
+ * 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.security.auth;
+
+import java.security.Principal;
+
+public class NimbusPrincipal implements Principal {
+
+  @Override
+  public String getName() {
+    return NimbusPrincipal.class.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/utils/BufferInputStream.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/BufferInputStream.java b/storm-core/src/jvm/backtype/storm/utils/BufferInputStream.java
new file mode 100644
index 0000000..c718858
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/utils/BufferInputStream.java
@@ -0,0 +1,53 @@
+/**
+ * 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.utils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+
+
+public class BufferInputStream {
+    byte[] buffer;
+    InputStream stream;
+
+    public BufferInputStream(InputStream stream, int bufferSize) {
+        this.stream = stream;
+        buffer = new byte[bufferSize];
+    }
+
+    public BufferInputStream(InputStream stream) {
+        this(stream, 15*1024);
+    }
+
+    public byte[] read() throws IOException {
+        int length = stream.read(buffer);
+        if(length==-1) {
+            close();
+            return new byte[0];
+        } else if(length==buffer.length) {
+            return buffer;
+        } else {
+            return Arrays.copyOf(buffer, length);
+        }
+    }
+
+    public void close() throws IOException {
+        stream.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/utils/ShellUtils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/ShellUtils.java b/storm-core/src/jvm/backtype/storm/utils/ShellUtils.java
index 8595b71..55e3866 100644
--- a/storm-core/src/jvm/backtype/storm/utils/ShellUtils.java
+++ b/storm-core/src/jvm/backtype/storm/utils/ShellUtils.java
@@ -102,6 +102,13 @@ abstract public class ShellUtils {
         this(interval, false);
     }
 
+    /** get the exit code
+     * @return the exit code of the process
+     */
+    public int getExitCode() {
+      return exitCode;
+    }
+
     /**
      * @param interval the minimum duration to wait before re-executing the
      *        command.

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/utils/Utils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/Utils.java b/storm-core/src/jvm/backtype/storm/utils/Utils.java
index c086be2..e0bbb1f 100644
--- a/storm-core/src/jvm/backtype/storm/utils/Utils.java
+++ b/storm-core/src/jvm/backtype/storm/utils/Utils.java
@@ -18,18 +18,35 @@
 package backtype.storm.utils;
 
 import backtype.storm.Config;
+import backtype.storm.blobstore.BlobStore;
+import backtype.storm.blobstore.BlobStoreAclHandler;
+import backtype.storm.blobstore.ClientBlobStore;
+import backtype.storm.blobstore.InputStreamWithMeta;
+import backtype.storm.blobstore.LocalFsBlobStore;
+import backtype.storm.generated.AccessControl;
+import backtype.storm.generated.AccessControlType;
 import backtype.storm.generated.AuthorizationException;
 import backtype.storm.generated.ComponentCommon;
 import backtype.storm.generated.ComponentObject;
+import backtype.storm.generated.KeyNotFoundException;
+import backtype.storm.generated.ReadableBlobMeta;
+import backtype.storm.generated.SettableBlobMeta;
 import backtype.storm.generated.StormTopology;
+import backtype.storm.localizer.Localizer;
+import backtype.storm.nimbus.NimbusInfo;
 import backtype.storm.serialization.DefaultSerializationDelegate;
 import backtype.storm.serialization.SerializationDelegate;
 import clojure.lang.IFn;
 import clojure.lang.RT;
+import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
+import org.apache.commons.compress.archivers.tar.TarArchiveInputStream;
 import org.apache.commons.lang.StringUtils;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.thrift.TBase;
+import org.apache.thrift.TDeserializer;
 import org.apache.thrift.TException;
+import org.apache.thrift.TSerializer;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Id;
@@ -39,13 +56,16 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.yaml.snakeyaml.Yaml;
 import org.yaml.snakeyaml.constructor.SafeConstructor;
-
 import java.net.URL;
 import java.net.URLDecoder;
 import java.nio.ByteBuffer;
-import java.nio.channels.Channels;
-import java.nio.channels.WritableByteChannel;
+import java.nio.file.FileSystems;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.attribute.PosixFilePermission;
+
 import java.io.File;
+import java.io.FileReader;
 import java.io.FileInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.ObjectOutputStream;
@@ -57,13 +77,21 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.FileOutputStream;
 import java.io.BufferedReader;
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.PrintStream;
 import java.io.RandomAccessFile;
 import java.io.Serializable;
 import java.io.IOException;
+
+import java.util.jar.JarEntry;
+import java.util.jar.JarFile;
 import java.util.Map;
+import java.util.Set;
 import java.util.Iterator;
 import java.util.Enumeration;
 import java.util.List;
+import java.util.Arrays;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.HashMap;
@@ -83,6 +111,9 @@ import org.apache.thrift.TSerializer;
 public class Utils {
     private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
     public static final String DEFAULT_STREAM_ID = "default";
+    public static final String DEFAULT_BLOB_VERSION_SUFFIX = ".version";
+    public static final String CURRENT_BLOB_SUFFIX_ID = "current";
+    public static final String DEFAULT_CURRENT_BLOB_SUFFIX = "." + CURRENT_BLOB_SUFFIX_ID;
     private static ThreadLocal<TSerializer> threadSer = new ThreadLocal<TSerializer>();
     private static ThreadLocal<TDeserializer> threadDes = new ThreadLocal<TDeserializer>();
 
@@ -110,28 +141,6 @@ public class Utils {
         return serializationDelegate.deserialize(serialized, clazz);
     }
 
-    public static byte[] thriftSerialize(TBase t) {
-        try {
-            TSerializer ser = threadSer.get();
-            if (ser == null) {
-                ser = new TSerializer();
-                threadSer.set(ser);
-            } 
-            return ser.serialize(t);
-        } catch (TException e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    private static TDeserializer getDes() {
-        TDeserializer des = threadDes.get();
-        if(des == null) {
-            des = new TDeserializer();
-            threadDes.set(des);
-        }
-        return des;
-    }
-
     public static <T> T thriftDeserialize(Class c, byte[] b, int offset, int length) {
         try {
             T ret = (T) c.newInstance();
@@ -143,18 +152,6 @@ public class Utils {
         }
     }
 
-    public static <T> T thriftDeserialize(Class c, byte[] b) {
-        try {
-            T ret = (T) c.newInstance();
-            TDeserializer des = getDes();
-            des.deserialize((TBase) ret, b);
-            return ret;
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
-
-    }
-    
     public static byte[] javaSerialize(Object obj) {
         try {
             ByteArrayOutputStream bos = new ByteArrayOutputStream();
@@ -431,38 +428,170 @@ public class Utils {
         return ret;
     }
 
-    public static void downloadFromMaster(Map conf, String file, String localFile) throws AuthorizationException, IOException, TException {
-        NimbusClient client = NimbusClient.getConfiguredClient(conf);
-        try {
-            download(client, file, localFile);
-        } finally {
-            client.close();
+
+    public static Localizer createLocalizer(Map conf, String baseDir) {
+        return new Localizer(conf, baseDir);
+    }
+
+    public static ClientBlobStore getClientBlobStoreForSupervisor(Map conf) {
+        ClientBlobStore store = (ClientBlobStore) newInstance(
+                (String) conf.get(Config.SUPERVISOR_BLOBSTORE));
+        store.prepare(conf);
+        return store;
+    }
+
+    public static BlobStore getNimbusBlobStore(Map conf, NimbusInfo nimbusInfo) {
+        return getNimbusBlobStore(conf, null, nimbusInfo);
+    }
+
+    public static BlobStore getNimbusBlobStore(Map conf, String baseDir, NimbusInfo nimbusInfo) {
+        String type = (String)conf.get(Config.NIMBUS_BLOBSTORE);
+        if (type == null) {
+            type = LocalFsBlobStore.class.getName();
         }
+        BlobStore store = (BlobStore) newInstance(type);
+        HashMap nconf = new HashMap(conf);
+        // only enable cleanup of blobstore on nimbus
+        nconf.put(Config.BLOBSTORE_CLEANUP_ENABLE, Boolean.TRUE);
+        store.prepare(nconf, baseDir, nimbusInfo);
+        return store;
     }
 
-    public static void downloadFromHost(Map conf, String file, String localFile, String host, int port) throws IOException, TException, AuthorizationException {
-        NimbusClient client = new NimbusClient (conf, host, port, null);
-        try {
-            download(client, file, localFile);
-        } finally {
-            client.close();
+    /**
+     * Meant to be called only by the supervisor for stormjar/stormconf/stormcode files.
+     * @param key
+     * @param localFile
+     * @param cb
+     * @throws AuthorizationException
+     * @throws KeyNotFoundException
+     * @throws IOException
+     */
+    public static void downloadResourcesAsSupervisor(String key, String localFile,
+                                                     ClientBlobStore cb) throws AuthorizationException, KeyNotFoundException, IOException {
+        final int MAX_RETRY_ATTEMPTS = 2;
+        final int ATTEMPTS_INTERVAL_TIME = 100;
+        for (int retryAttempts = 0; retryAttempts < MAX_RETRY_ATTEMPTS; retryAttempts++) {
+            if (downloadResourcesAsSupervisorAttempt(cb, key, localFile)) {
+                break;
+            }
+            Utils.sleep(ATTEMPTS_INTERVAL_TIME);
         }
     }
 
-    private static void download(NimbusClient client, String file, String localFile) throws IOException, TException, AuthorizationException {
-        WritableByteChannel out = Channels.newChannel(new FileOutputStream(localFile));
+    public static ClientBlobStore getClientBlobStore(Map conf) {
+        ClientBlobStore store = (ClientBlobStore) Utils.newInstance((String) conf.get(Config.CLIENT_BLOBSTORE));
+        store.prepare(conf);
+        return store;
+    }
+
+    private static boolean downloadResourcesAsSupervisorAttempt(ClientBlobStore cb, String key, String localFile) {
+        boolean isSuccess = false;
+        FileOutputStream out = null;
+        InputStreamWithMeta in = null;
         try {
-            String id = client.getClient().beginFileDownload(file);
-            while (true) {
-                ByteBuffer chunk = client.getClient().downloadChunk(id);
-                int written = out.write(chunk);
-                if (written == 0) break;
+            out = new FileOutputStream(localFile);
+            in = cb.getBlob(key);
+            long fileSize = in.getFileLength();
+
+            byte[] buffer = new byte[1024];
+            int len;
+            int downloadFileSize = 0;
+            while ((len = in.read(buffer)) >= 0) {
+                out.write(buffer, 0, len);
+                downloadFileSize += len;
             }
+
+            isSuccess = (fileSize == downloadFileSize);
+        } catch (TException | IOException e) {
+            LOG.error("An exception happened while downloading {} from blob store.", localFile, e);
         } finally {
-            out.close();
+            try {
+                if (out != null) {
+                    out.close();
+                }
+            } catch (IOException ignored) {}
+            try {
+                if (in != null) {
+                    in.close();
+                }
+            } catch (IOException ignored) {}
+        }
+        if (!isSuccess) {
+            try {
+                Files.deleteIfExists(Paths.get(localFile));
+            } catch (IOException ex) {
+                LOG.error("Failed trying to delete the partially downloaded {}", localFile, ex);
+            }
+        }
+        return isSuccess;
+    }
+
+    public static boolean checkFileExists(String dir, String file) {
+        return Files.exists(new File(dir, file).toPath());
+    }
+
+    public static long nimbusVersionOfBlob(String key, ClientBlobStore cb) throws AuthorizationException, KeyNotFoundException {
+        long nimbusBlobVersion = 0;
+        ReadableBlobMeta metadata = cb.getBlobMeta(key);
+        nimbusBlobVersion = metadata.get_version();
+        return nimbusBlobVersion;
+    }
+
+    public static String getFileOwner(String path) throws IOException {
+        return Files.getOwner(FileSystems.getDefault().getPath(path)).getName();
+    }
+
+    public static long localVersionOfBlob(String localFile) {
+        File f = new File(localFile + DEFAULT_BLOB_VERSION_SUFFIX);
+        long currentVersion = 0;
+        if (f.exists() && !(f.isDirectory())) {
+            BufferedReader br = null;
+            try {
+                br = new BufferedReader(new FileReader(f));
+                String line = br.readLine();
+                currentVersion = Long.parseLong(line);
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            } finally {
+                try {
+                    if (br != null) {
+                        br.close();
+                    }
+                } catch (Exception ignore) {
+                    LOG.error("Exception trying to cleanup", ignore);
+                }
+            }
+            return currentVersion;
+        } else {
+            return -1;
+        }
+    }
+
+    public static String constructBlobWithVersionFileName(String fileName, long version) {
+        return fileName + "." + version;
+    }
+
+    public static String constructBlobCurrentSymlinkName(String fileName) {
+        return fileName + Utils.DEFAULT_CURRENT_BLOB_SUFFIX;
+    }
+
+    public static String constructVersionFileName(String fileName) {
+        return fileName + Utils.DEFAULT_BLOB_VERSION_SUFFIX;
+    }
+    // only works on operating  systems that support posix
+    public static void restrictPermissions(String baseDir) {
+        try {
+            Set<PosixFilePermission> perms = new HashSet<PosixFilePermission>(
+                    Arrays.asList(PosixFilePermission.OWNER_READ, PosixFilePermission.OWNER_WRITE,
+                            PosixFilePermission.OWNER_EXECUTE, PosixFilePermission.GROUP_READ,
+                            PosixFilePermission.GROUP_EXECUTE));
+            Files.setPosixFilePermissions(FileSystems.getDefault().getPath(baseDir), perms);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
         }
     }
 
+
     public static IFn loadClojureFn(String namespace, String name) {
         try {
             clojure.lang.Compiler.eval(RT.readString("(require '" + namespace + ")"));
@@ -505,6 +634,37 @@ public class Utils {
         return result;
     }
 
+    private static TDeserializer getDes() {
+        TDeserializer des = threadDes.get();
+        if(des == null) {
+            des = new TDeserializer();
+            threadDes.set(des);
+        }
+        return des;
+    }
+
+    public static byte[] thriftSerialize(TBase t) {
+        try {
+            TSerializer ser = threadSer.get();
+            if (ser == null) {
+                ser = new TSerializer();
+                threadSer.set(ser);
+            }
+            return ser.serialize(t);
+        } catch (TException e) {
+            LOG.error("Failed to serialize to thrift: ", e);
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static <T> T thriftDeserialize(Class c, byte[] b) {
+        try {
+            return Utils.thriftDeserialize(c, b, 0, b.length);
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
     public static Integer getInt(Object o, Integer defaultValue) {
         if (null == o) {
             return defaultValue;
@@ -571,6 +731,245 @@ public class Utils {
         return UUID.randomUUID().getLeastSignificantBits();
     }
 
+    /**
+     * Unpack matching files from a jar. Entries inside the jar that do
+     * not match the given pattern will be skipped.
+     *
+     * @param jarFile the .jar file to unpack
+     * @param toDir the destination directory into which to unpack the jar
+     */
+    public static void unJar(File jarFile, File toDir)
+            throws IOException {
+        JarFile jar = new JarFile(jarFile);
+        try {
+            Enumeration<JarEntry> entries = jar.entries();
+            while (entries.hasMoreElements()) {
+                final JarEntry entry = entries.nextElement();
+                if (!entry.isDirectory()) {
+                    InputStream in = jar.getInputStream(entry);
+                    try {
+                        File file = new File(toDir, entry.getName());
+                        ensureDirectory(file.getParentFile());
+                        OutputStream out = new FileOutputStream(file);
+                        try {
+                            copyBytes(in, out, 8192);
+                        } finally {
+                            out.close();
+                        }
+                    } finally {
+                        in.close();
+                    }
+                }
+            }
+        } finally {
+            jar.close();
+        }
+    }
+
+    /**
+     * Copies from one stream to another.
+     *
+     * @param in InputStream to read from
+     * @param out OutputStream to write to
+     * @param buffSize the size of the buffer
+     */
+    public static void copyBytes(InputStream in, OutputStream out, int buffSize)
+            throws IOException {
+        PrintStream ps = out instanceof PrintStream ? (PrintStream)out : null;
+        byte buf[] = new byte[buffSize];
+        int bytesRead = in.read(buf);
+        while (bytesRead >= 0) {
+            out.write(buf, 0, bytesRead);
+            if ((ps != null) && ps.checkError()) {
+                throw new IOException("Unable to write to output stream.");
+            }
+            bytesRead = in.read(buf);
+        }
+    }
+
+    /**
+     * Ensure the existence of a given directory.
+     *
+     * @throws IOException if it cannot be created and does not already exist
+     */
+    private static void ensureDirectory(File dir) throws IOException {
+        if (!dir.mkdirs() && !dir.isDirectory()) {
+            throw new IOException("Mkdirs failed to create " +
+                    dir.toString());
+        }
+    }
+
+    /**
+     * Given a Tar File as input it will untar the file in a the untar directory
+     * passed as the second parameter
+     * <p/>
+     * This utility will untar ".tar" files and ".tar.gz","tgz" files.
+     *
+     * @param inFile   The tar file as input.
+     * @param untarDir The untar directory where to untar the tar file.
+     * @throws IOException
+     */
+    public static void unTar(File inFile, File untarDir) throws IOException {
+        if (!untarDir.mkdirs()) {
+            if (!untarDir.isDirectory()) {
+                throw new IOException("Mkdirs failed to create " + untarDir);
+            }
+        }
+
+        boolean gzipped = inFile.toString().endsWith("gz");
+        if (onWindows()) {
+            // Tar is not native to Windows. Use simple Java based implementation for
+            // tests and simple tar archives
+            unTarUsingJava(inFile, untarDir, gzipped);
+        } else {
+            // spawn tar utility to untar archive for full fledged unix behavior such
+            // as resolving symlinks in tar archives
+            unTarUsingTar(inFile, untarDir, gzipped);
+        }
+    }
+
+    private static void unTarUsingTar(File inFile, File untarDir,
+                                      boolean gzipped) throws IOException {
+        StringBuffer untarCommand = new StringBuffer();
+        if (gzipped) {
+            untarCommand.append(" gzip -dc '");
+            untarCommand.append(inFile.toString());
+            untarCommand.append("' | (");
+        }
+        untarCommand.append("cd '");
+        untarCommand.append(untarDir.toString());
+        untarCommand.append("' ; ");
+        untarCommand.append("tar -xf ");
+
+        if (gzipped) {
+            untarCommand.append(" -)");
+        } else {
+            untarCommand.append(inFile.toString());
+        }
+        String[] shellCmd = {"bash", "-c", untarCommand.toString()};
+        ShellUtils.ShellCommandExecutor shexec = new ShellUtils.ShellCommandExecutor(shellCmd);
+        shexec.execute();
+        int exitcode = shexec.getExitCode();
+        if (exitcode != 0) {
+            throw new IOException("Error untarring file " + inFile +
+                    ". Tar process exited with exit code " + exitcode);
+        }
+    }
+
+    private static void unTarUsingJava(File inFile, File untarDir,
+                                       boolean gzipped) throws IOException {
+        InputStream inputStream = null;
+        TarArchiveInputStream tis = null;
+        try {
+            if (gzipped) {
+                inputStream = new BufferedInputStream(new GZIPInputStream(
+                        new FileInputStream(inFile)));
+            } else {
+                inputStream = new BufferedInputStream(new FileInputStream(inFile));
+            }
+            tis = new TarArchiveInputStream(inputStream);
+            for (TarArchiveEntry entry = tis.getNextTarEntry(); entry != null; ) {
+                unpackEntries(tis, entry, untarDir);
+                entry = tis.getNextTarEntry();
+            }
+        } finally {
+            cleanup(tis, inputStream);
+        }
+    }
+
+    /**
+     * Close the Closeable objects and <b>ignore</b> any {@link IOException} or
+     * null pointers. Must only be used for cleanup in exception handlers.
+     *
+     * @param closeables the objects to close
+     */
+    private static void cleanup(java.io.Closeable... closeables) {
+        for (java.io.Closeable c : closeables) {
+            if (c != null) {
+                try {
+                    c.close();
+                } catch (IOException e) {
+                    LOG.debug("Exception in closing " + c, e);
+
+                }
+            }
+        }
+    }
+
+    private static void unpackEntries(TarArchiveInputStream tis,
+                                      TarArchiveEntry entry, File outputDir) throws IOException {
+        if (entry.isDirectory()) {
+            File subDir = new File(outputDir, entry.getName());
+            if (!subDir.mkdirs() && !subDir.isDirectory()) {
+                throw new IOException("Mkdirs failed to create tar internal dir "
+                        + outputDir);
+            }
+            for (TarArchiveEntry e : entry.getDirectoryEntries()) {
+                unpackEntries(tis, e, subDir);
+            }
+            return;
+        }
+        File outputFile = new File(outputDir, entry.getName());
+        if (!outputFile.getParentFile().exists()) {
+            if (!outputFile.getParentFile().mkdirs()) {
+                throw new IOException("Mkdirs failed to create tar internal dir "
+                        + outputDir);
+            }
+        }
+        int count;
+        byte data[] = new byte[2048];
+        BufferedOutputStream outputStream = new BufferedOutputStream(
+                new FileOutputStream(outputFile));
+
+        while ((count = tis.read(data)) != -1) {
+            outputStream.write(data, 0, count);
+        }
+        outputStream.flush();
+        outputStream.close();
+    }
+
+    public static boolean onWindows() {
+        if (System.getenv("OS") != null) {
+            return System.getenv("OS").equals("Windows_NT");
+        }
+        return false;
+    }
+
+    public static void unpack(File localrsrc, File dst) throws IOException {
+        String lowerDst = localrsrc.getName().toLowerCase();
+        if (lowerDst.endsWith(".jar")) {
+            unJar(localrsrc, dst);
+        } else if (lowerDst.endsWith(".zip")) {
+            unZip(localrsrc, dst);
+        } else if (lowerDst.endsWith(".tar.gz") ||
+                lowerDst.endsWith(".tgz") ||
+                lowerDst.endsWith(".tar")) {
+            unTar(localrsrc, dst);
+        } else {
+            LOG.warn("Cannot unpack " + localrsrc);
+            if (!localrsrc.renameTo(dst)) {
+                throw new IOException("Unable to rename file: [" + localrsrc
+                        + "] to [" + dst + "]");
+            }
+        }
+        if (localrsrc.isFile()) {
+            localrsrc.delete();
+        }
+    }
+
+    public static boolean canUserReadBlob(ReadableBlobMeta meta, String user) {
+        SettableBlobMeta settable = meta.get_settable();
+        for (AccessControl acl : settable.get_acl()) {
+            if (acl.get_type().equals(AccessControlType.OTHER) && (acl.get_access() & BlobStoreAclHandler.READ) > 0) {
+                return true;
+            }
+            if (acl.get_name().equals(user) && (acl.get_access() & BlobStoreAclHandler.READ) > 0) {
+                return true;
+            }
+        }
+        return false;
+    }
+
     public static CuratorFramework newCurator(Map conf, List<String> servers, Object port, String root) {
         return newCurator(conf, servers, port, root, null);
     }
@@ -702,6 +1101,38 @@ public class Utils {
         return ret;
     }
 
+    /**
+     * Takes an input dir or file and returns the disk usage on that local directory.
+     * Very basic implementation.
+     *
+     * @param dir The input dir to get the disk space of this local dir
+     * @return The total disk space of the input local directory
+     */
+    public static long getDU(File dir) {
+        long size = 0;
+        if (!dir.exists())
+            return 0;
+        if (!dir.isDirectory()) {
+            return dir.length();
+        } else {
+            File[] allFiles = dir.listFiles();
+            if(allFiles != null) {
+                for (int i = 0; i < allFiles.length; i++) {
+                    boolean isSymLink;
+                    try {
+                        isSymLink = org.apache.commons.io.FileUtils.isSymlink(allFiles[i]);
+                    } catch(IOException ioe) {
+                        isSymLink = true;
+                    }
+                    if(!isSymLink) {
+                        size += getDU(allFiles[i]);
+                    }
+                }
+            }
+            return size;
+        }
+    }
+
     public static String threadDump() {
         final StringBuilder dump = new StringBuilder();
         final java.lang.management.ThreadMXBean threadMXBean =  java.lang.management.ManagementFactory.getThreadMXBean();

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java b/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java
index 44ec967..f1be007 100644
--- a/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java
+++ b/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java
@@ -477,8 +477,16 @@ public class ConfigValidation {
         }
     }
 
-    public static class PacemakerAuthTypeValidator extends Validator {
+    public static class MapOfStringToMapOfStringToObjectValidator extends Validator {
+      @Override
+      public  void validateField(String name, Object o) {
+        ConfigValidationUtils.NestableFieldValidator validator = ConfigValidationUtils.mapFv(ConfigValidationUtils.fv(String.class, false),
+                ConfigValidationUtils.mapFv(String.class, Object.class,true), true);
+        validator.validateField(name, o);
+      }
+    }
 
+    public static class PacemakerAuthTypeValidator extends Validator {
         @Override
         public void validateField(String name, Object o) {
             if(o == null) {
@@ -486,9 +494,9 @@ public class ConfigValidation {
             }
 
             if(o instanceof String &&
-               (((String)o).equals("NONE") ||
-                ((String)o).equals("DIGEST") ||
-                ((String)o).equals("KERBEROS"))) {
+                    (((String)o).equals("NONE") ||
+                            ((String)o).equals("DIGEST") ||
+                            ((String)o).equals("KERBEROS"))) {
                 return;
             }
             throw new IllegalArgumentException( "Field " + name + " must be one of \"NONE\", \"DIGEST\", or \"KERBEROS\"");

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/validation/ConfigValidationAnnotations.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/validation/ConfigValidationAnnotations.java b/storm-core/src/jvm/backtype/storm/validation/ConfigValidationAnnotations.java
index ed93370..cb742fa 100644
--- a/storm-core/src/jvm/backtype/storm/validation/ConfigValidationAnnotations.java
+++ b/storm-core/src/jvm/backtype/storm/validation/ConfigValidationAnnotations.java
@@ -51,7 +51,6 @@ public class ConfigValidationAnnotations {
     /**
      * Validators with fields: validatorClass and type
      */
-
     @Retention(RetentionPolicy.RUNTIME)
     @Target(ElementType.FIELD)
     public @interface isType {
@@ -82,7 +81,6 @@ public class ConfigValidationAnnotations {
     /**
      * Validators with fields: validatorClass
      */
-
     @Retention(RetentionPolicy.RUNTIME)
     @Target(ElementType.FIELD)
     public @interface isString {
@@ -109,7 +107,7 @@ public class ConfigValidationAnnotations {
     }
 
     /**
-     * validates on object is not null
+     * Validates on object is not null
      */
     @Retention(RetentionPolicy.RUNTIME)
     @Target(ElementType.FIELD)
@@ -118,7 +116,7 @@ public class ConfigValidationAnnotations {
     }
 
     /**
-     * validates that there are no duplicates in a list
+     * Validates that there are no duplicates in a list
      */
     @Retention(RetentionPolicy.RUNTIME)
     @Target(ElementType.FIELD)
@@ -142,7 +140,6 @@ public class ConfigValidationAnnotations {
      * Validates the type of each key and value in a map
      * Validator with fields: validatorClass, keyValidatorClass, valueValidatorClass
      */
-
     @Retention(RetentionPolicy.RUNTIME)
     @Target(ElementType.FIELD)
     public @interface isMapEntryType {
@@ -168,7 +165,7 @@ public class ConfigValidationAnnotations {
     }
 
     /**
-     * checks if a number is positive and whether zero inclusive
+     * Checks if a number is positive and whether zero inclusive
      * Validator with fields: validatorClass, includeZero
      */
     @Retention(RetentionPolicy.RUNTIME)
@@ -182,7 +179,6 @@ public class ConfigValidationAnnotations {
     /**
      * Complex/custom type validators
      */
-
     @Retention(RetentionPolicy.RUNTIME)
     @Target(ElementType.FIELD)
     public @interface isStringOrStringList {
@@ -204,7 +200,6 @@ public class ConfigValidationAnnotations {
     /**
      * For custom validators
      */
-
     @Retention(RetentionPolicy.RUNTIME)
     @Target(ElementType.FIELD)
     public @interface CustomValidator {

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/py/storm/Nimbus-remote
----------------------------------------------------------------------
diff --git a/storm-core/src/py/storm/Nimbus-remote b/storm-core/src/py/storm/Nimbus-remote
index 63e7dce..5b8e396 100644
--- a/storm-core/src/py/storm/Nimbus-remote
+++ b/storm-core/src/py/storm/Nimbus-remote
@@ -55,6 +55,20 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  void setWorkerProfiler(string id, ProfileRequest profileRequest)')
   print('   getComponentPendingProfileActions(string id, string component_id, ProfileAction action)')
   print('  void uploadNewCredentials(string name, Credentials creds)')
+  print('  string beginCreateBlob(string key, SettableBlobMeta meta)')
+  print('  string beginUpdateBlob(string key)')
+  print('  void uploadBlobChunk(string session, string chunk)')
+  print('  void finishBlobUpload(string session)')
+  print('  void cancelBlobUpload(string session)')
+  print('  ReadableBlobMeta getBlobMeta(string key)')
+  print('  void setBlobMeta(string key, SettableBlobMeta meta)')
+  print('  BeginDownloadResult beginBlobDownload(string key)')
+  print('  string downloadBlobChunk(string session)')
+  print('  void deleteBlob(string key)')
+  print('  ListBlobsResult listBlobs(string session)')
+  print('  i32 getBlobReplication(string key)')
+  print('  i32 updateBlobReplication(string key, i32 replication)')
+  print('  void createStateInZookeeper(string key)')
   print('  string beginFileUpload()')
   print('  void uploadChunk(string location, string chunk)')
   print('  void finishFileUpload(string location)')
@@ -204,6 +218,90 @@ elif cmd == 'uploadNewCredentials':
     sys.exit(1)
   pp.pprint(client.uploadNewCredentials(args[0],eval(args[1]),))
 
+elif cmd == 'beginCreateBlob':
+  if len(args) != 2:
+    print('beginCreateBlob requires 2 args')
+    sys.exit(1)
+  pp.pprint(client.beginCreateBlob(args[0],eval(args[1]),))
+
+elif cmd == 'beginUpdateBlob':
+  if len(args) != 1:
+    print('beginUpdateBlob requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.beginUpdateBlob(args[0],))
+
+elif cmd == 'uploadBlobChunk':
+  if len(args) != 2:
+    print('uploadBlobChunk requires 2 args')
+    sys.exit(1)
+  pp.pprint(client.uploadBlobChunk(args[0],args[1],))
+
+elif cmd == 'finishBlobUpload':
+  if len(args) != 1:
+    print('finishBlobUpload requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.finishBlobUpload(args[0],))
+
+elif cmd == 'cancelBlobUpload':
+  if len(args) != 1:
+    print('cancelBlobUpload requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.cancelBlobUpload(args[0],))
+
+elif cmd == 'getBlobMeta':
+  if len(args) != 1:
+    print('getBlobMeta requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.getBlobMeta(args[0],))
+
+elif cmd == 'setBlobMeta':
+  if len(args) != 2:
+    print('setBlobMeta requires 2 args')
+    sys.exit(1)
+  pp.pprint(client.setBlobMeta(args[0],eval(args[1]),))
+
+elif cmd == 'beginBlobDownload':
+  if len(args) != 1:
+    print('beginBlobDownload requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.beginBlobDownload(args[0],))
+
+elif cmd == 'downloadBlobChunk':
+  if len(args) != 1:
+    print('downloadBlobChunk requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.downloadBlobChunk(args[0],))
+
+elif cmd == 'deleteBlob':
+  if len(args) != 1:
+    print('deleteBlob requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.deleteBlob(args[0],))
+
+elif cmd == 'listBlobs':
+  if len(args) != 1:
+    print('listBlobs requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.listBlobs(args[0],))
+
+elif cmd == 'getBlobReplication':
+  if len(args) != 1:
+    print('getBlobReplication requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.getBlobReplication(args[0],))
+
+elif cmd == 'updateBlobReplication':
+  if len(args) != 2:
+    print('updateBlobReplication requires 2 args')
+    sys.exit(1)
+  pp.pprint(client.updateBlobReplication(args[0],eval(args[1]),))
+
+elif cmd == 'createStateInZookeeper':
+  if len(args) != 1:
+    print('createStateInZookeeper requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.createStateInZookeeper(args[0],))
+
 elif cmd == 'beginFileUpload':
   if len(args) != 0:
     print('beginFileUpload requires 0 args')


[09/17] storm git commit: Blobstore API STORM- 876

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/blobstore/BlobStoreUtils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/blobstore/BlobStoreUtils.java b/storm-core/src/jvm/backtype/storm/blobstore/BlobStoreUtils.java
new file mode 100644
index 0000000..5e39743
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/blobstore/BlobStoreUtils.java
@@ -0,0 +1,257 @@
+/**
+ * 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);
+            int currentSeqNumber = Integer.parseInt(sequenceInfo.getSequenceNumber());
+            if (seqNumber < currentSeqNumber) {
+                seqNumber = currentSeqNumber;
+                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;
+        ReadableBlobMeta rbm;
+        ClientBlobStore remoteBlobStore;
+        InputStreamWithMeta in;
+        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 isSuccess;
+    }
+
+    // Download updated blobs from potential nimbodes
+    public static boolean downloadUpdatedBlob(Map conf, BlobStore blobStore, String key, Set<NimbusInfo> nimbusInfos)
+            throws TTransportException {
+        NimbusClient client;
+        ClientBlobStore remoteBlobStore;
+        InputStreamWithMeta in;
+        AtomicOutputStream out;
+        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 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>();
+        if (keys != null) {
+            while (keys.hasNext()) {
+                keyList.add(keys.next());
+            }
+        }
+        LOG.debug("KeyList from blobstore {}", keyList);
+        return keyList;
+    }
+
+    public static void createStateInZookeeper(Map conf, String key, NimbusInfo nimbusInfo) throws TTransportException {
+        ClientBlobStore cb = new NimbusBlobStore();
+        cb.setClient(conf, new NimbusClient(conf, nimbusInfo.getHost(), nimbusInfo.getPort(), null));
+        cb.createStateInZookeeper(key);
+    }
+
+    public static void updateKeyForBlobStore (Map conf, BlobStore blobStore, CuratorFramework zkClient, String key, NimbusInfo nimbusDetails) {
+        try {
+            // Most of clojure tests currently try to access the blobs using getBlob. Since, updateKeyForBlobStore
+            // checks for updating the correct version of the blob as a part of nimbus ha before performing any
+            // operation on it, there is a neccessity to stub several test cases to ignore this method. It is a valid
+            // trade off to return if nimbusDetails which include the details of the current nimbus host port data are
+            // not initialized as a part of the test. Moreover, this applies to only local blobstore when used along with
+            // nimbus ha.
+            if (nimbusDetails == null) {
+                return;
+            }
+            boolean isListContainsCurrentNimbusInfo = false;
+            List<String> stateInfo;
+            if (zkClient.checkExists().forPath(BLOBSTORE_SUBTREE + "/" + key) == null) {
+                return;
+            }
+            stateInfo = zkClient.getChildren().forPath(BLOBSTORE_SUBTREE + "/" + key);
+            LOG.debug("StateInfo for update {}", stateInfo);
+            Set<NimbusInfo> nimbusInfoList = getNimbodesWithLatestSequenceNumberOfBlob(zkClient, key);
+
+            for (NimbusInfo nimbusInfo:nimbusInfoList) {
+                if (nimbusInfo.getHost().equals(nimbusDetails.getHost())) {
+                    isListContainsCurrentNimbusInfo = true;
+                    break;
+                }
+            }
+
+            if (!isListContainsCurrentNimbusInfo && downloadUpdatedBlob(conf, blobStore, key, nimbusInfoList)) {
+                LOG.debug("Updating state inside zookeeper for an update");
+                createStateInZookeeper(conf, key, nimbusDetails);
+            }
+        } catch (Exception exp) {
+            throw new RuntimeException(exp);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/blobstore/BlobSynchronizer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/blobstore/BlobSynchronizer.java b/storm-core/src/jvm/backtype/storm/blobstore/BlobSynchronizer.java
new file mode 100644
index 0000000..abd7c86
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/blobstore/BlobSynchronizer.java
@@ -0,0 +1,124 @@
+/**
+ * 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 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);
+            }
+        } catch (Exception exp) {
+            throw new RuntimeException(exp);
+        }
+    }
+
+    // Make a key list to download
+    public Set<String> getKeySetToDownload(Set<String> blobStoreKeySet, Set<String> zookeeperKeySet) {
+        zookeeperKeySet.removeAll(blobStoreKeySet);
+        LOG.debug("Key list to download {}", zookeeperKeySet);
+        return zookeeperKeySet;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/blobstore/ClientBlobStore.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/blobstore/ClientBlobStore.java b/storm-core/src/jvm/backtype/storm/blobstore/ClientBlobStore.java
new file mode 100644
index 0000000..cc40aff
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/blobstore/ClientBlobStore.java
@@ -0,0 +1,62 @@
+/**
+ * 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.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 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 int getBlobReplication(String Key) throws AuthorizationException, KeyNotFoundException;
+    public abstract int updateBlobReplication(String Key, int replication) throws AuthorizationException, KeyNotFoundException;
+    public abstract boolean setClient(Map conf, NimbusClient client);
+    public abstract void createStateInZookeeper(String key);
+
+    public final AtomicOutputStream createBlob(String key, SettableBlobMeta meta) throws AuthorizationException, KeyAlreadyExistsException {
+        if (meta !=null && meta.is_set_acl()) {
+            BlobStoreAclHandler.validateSettableACLs(key, meta.get_acl());
+        }
+        return createBlobToExtend(key, meta);
+    }
+
+    public final void setBlobMeta(String key, SettableBlobMeta meta) throws AuthorizationException, KeyNotFoundException {
+        if (meta !=null && meta.is_set_acl()) {
+            BlobStoreAclHandler.validateSettableACLs(key, meta.get_acl());
+        }
+        setBlobMetaToExtend(key, meta);
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/blobstore/FileBlobStoreImpl.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/blobstore/FileBlobStoreImpl.java b/storm-core/src/jvm/backtype/storm/blobstore/FileBlobStoreImpl.java
new file mode 100644
index 0000000..b789335
--- /dev/null
+++ b/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) {
+            for (File sub: files) {
+                try {
+                    ret.add(new LocalFsBlobStoreFile(sub.getParentFile(), sub.getName()));
+                } catch (IllegalArgumentException e) {
+                    //Ignored the file did not match
+                    LOG.warn("Found an unexpected file in {} {}",path, sub.getName());
+                }
+            }
+        }
+        return ret.iterator();
+    }
+
+    protected Iterator<String> listKeys(File path) throws IOException {
+        String[] files = path.list();
+        if (files != null) {
+            return Arrays.asList(files).iterator();
+        }
+        return new LinkedList<String>().iterator();
+    }
+
+    protected void delete(File path) throws IOException {
+        Files.deleteIfExists(path.toPath());
+    }
+
+    public void shutdown() {
+        if (cleanup != null) {
+            cleanup.cancel();
+            cleanup = null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/blobstore/InputStreamWithMeta.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/blobstore/InputStreamWithMeta.java b/storm-core/src/jvm/backtype/storm/blobstore/InputStreamWithMeta.java
new file mode 100644
index 0000000..1d29fda
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/blobstore/InputStreamWithMeta.java
@@ -0,0 +1,26 @@
+/**
+ * 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.IOException;
+import java.io.InputStream;
+
+public abstract class InputStreamWithMeta extends InputStream {
+    public abstract long getVersion() throws IOException;
+    public abstract long getFileLength() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/blobstore/KeyFilter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/blobstore/KeyFilter.java b/storm-core/src/jvm/backtype/storm/blobstore/KeyFilter.java
new file mode 100644
index 0000000..32bb9fd
--- /dev/null
+++ b/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> {
+    R filter(String key);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/blobstore/KeySequenceNumber.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/blobstore/KeySequenceNumber.java b/storm-core/src/jvm/backtype/storm/blobstore/KeySequenceNumber.java
new file mode 100644
index 0000000..1cddac0
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/blobstore/KeySequenceNumber.java
@@ -0,0 +1,229 @@
+/**
+ * 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 latter 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;
+    private final int INITIAL_SEQUENCE_NUMBER = 1;
+
+    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(INITIAL_SEQUENCE_NUMBER).array());
+                return INITIAL_SEQUENCE_NUMBER;
+            }
+
+            // 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.normalizeNimbusHostPortSequenceNumberInfo(stateInfo)
+                        .getSequenceNumber()));
+            }
+
+            // 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 INITIAL_SEQUENCE_NUMBER - 1;
+                }
+            }
+
+            // It covers scenarios 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 + 1).array());
+    }
+
+    private int getMaxSequenceNumber(CuratorFramework zkClient) throws Exception {
+        return ByteBuffer.wrap(zkClient.getData()
+                .forPath(BLOBSTORE_MAX_KEY_SEQUENCE_SUBTREE + "/" + key)).getInt();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStore.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStore.java b/storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStore.java
new file mode 100644
index 0000000..0941b9a
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStore.java
@@ -0,0 +1,308 @@
+/**
+ * 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 final int allPermissions = READ | WRITE | ADMIN;
+    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, allPermissions);
+        BlobStoreAclHandler.validateSettableACLs(key, meta.get_acl());
+        _aclHandler.hasPermissions(meta.get_acl(), allPermissions, 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 {
+        CuratorFramework zkClient = null;
+        int replicationCount = 0;
+        try {
+            validateKey(key);
+            SettableBlobMeta meta = getStoredBlobMeta(key);
+            _aclHandler.hasPermissions(meta.get_acl(), READ, who, key);
+            zkClient = BlobStoreUtils.createZKClient(conf);
+            if (zkClient.checkExists().forPath(BLOBSTORE_SUBTREE + key) == null) {
+                zkClient.close();
+                return 0;
+            }
+            replicationCount = zkClient.getChildren().forPath(BLOBSTORE_SUBTREE + key).size();
+        } finally {
+            if (zkClient != null) {
+                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;
+        CuratorFramework zkClient = null;
+        try {
+            List<String> keyList = BlobStoreUtils.getKeyListFromBlobStore(this);
+            if (!keyList.contains(key)) {
+                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;
+                    }
+                }
+            }
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        } finally {
+            if (zkClient != null) {
+                zkClient.close();
+            }
+        }
+        return checkBlobDownload;
+    }
+
+    public void checkForBlobUpdate(String key) {
+        CuratorFramework zkClient = BlobStoreUtils.createZKClient(conf);
+        BlobStoreUtils.updateKeyForBlobStore(conf, this, zkClient, key, nimbusInfo);
+        zkClient.close();
+    }
+
+    public void fullCleanup(long age) throws IOException {
+        fbs.fullCleanup(age);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStoreFile.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStoreFile.java b/storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStoreFile.java
new file mode 100644
index 0000000..fb11fa6
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStoreFile.java
@@ -0,0 +1,159 @@
+/**
+ * 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.generated.SettableBlobMeta;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.regex.Matcher;
+
+public class LocalFsBlobStoreFile extends BlobStoreFile {
+
+    private final String _key;
+    private final boolean _isTmp;
+    private final File _path;
+    private Long _modTime = null;
+    private final boolean _mustBeNew;
+    private SettableBlobMeta meta;
+
+    public LocalFsBlobStoreFile(File base, String name) {
+        if (BlobStoreFile.BLOBSTORE_DATA_FILE.equals(name)) {
+            _isTmp = false;
+        } else {
+            Matcher m = TMP_NAME_PATTERN.matcher(name);
+            if (!m.matches()) {
+                throw new IllegalArgumentException("File name does not match '"+name+"' !~ "+TMP_NAME_PATTERN);
+            }
+            _isTmp = true;
+        }
+        _key = base.getName();
+        _path = new File(base, name);
+        _mustBeNew = false;
+    }
+
+    public LocalFsBlobStoreFile(File base, boolean isTmp, boolean mustBeNew) {
+        _key = base.getName();
+        _isTmp = isTmp;
+        _mustBeNew = mustBeNew;
+        if (_isTmp) {
+            _path = new File(base, System.currentTimeMillis()+TMP_EXT);
+        } else {
+            _path = new File(base, BlobStoreFile.BLOBSTORE_DATA_FILE);
+        }
+    }
+
+    @Override
+    public void delete() throws IOException {
+        _path.delete();
+    }
+
+    @Override
+    public boolean isTmp() {
+        return _isTmp;
+    }
+
+    @Override
+    public String getKey() {
+        return _key;
+    }
+
+    @Override
+    public long getModTime() throws IOException {
+        if (_modTime == null) {
+            _modTime = _path.lastModified();
+        }
+        return _modTime;
+    }
+
+    @Override
+    public InputStream getInputStream() throws IOException {
+        if (isTmp()) {
+            throw new IllegalStateException("Cannot read from a temporary part file.");
+        }
+        return new FileInputStream(_path);
+    }
+
+    @Override
+    public OutputStream getOutputStream() throws IOException {
+        if (!isTmp()) {
+            throw new IllegalStateException("Can only write to a temporary part file.");
+        }
+        boolean success = false;
+        try {
+            success = _path.createNewFile();
+        } catch (IOException e) {
+            //Try to create the parent directory, may not work
+            _path.getParentFile().mkdirs();
+            success = _path.createNewFile();
+        }
+        if (!success) {
+            throw new IOException(_path+" already exists");
+        }
+        return new FileOutputStream(_path);
+    }
+
+    @Override
+    public void commit() throws IOException {
+        if (!isTmp()) {
+            throw new IllegalStateException("Can only write to a temporary part file.");
+        }
+
+        File dest = new File(_path.getParentFile(), BlobStoreFile.BLOBSTORE_DATA_FILE);
+        if (_mustBeNew) {
+            Files.move(_path.toPath(), dest.toPath(), StandardCopyOption.ATOMIC_MOVE);
+        } else {
+            Files.move(_path.toPath(), dest.toPath(), StandardCopyOption.ATOMIC_MOVE, StandardCopyOption.REPLACE_EXISTING);
+        }
+    }
+
+    @Override
+    public void cancel() throws IOException {
+        if (!isTmp()) {
+            throw new IllegalStateException("Can only write to a temporary part file.");
+        }
+        delete();
+    }
+
+    @Override
+    public SettableBlobMeta getMetadata () {
+        return meta;
+    }
+
+    @Override
+    public void setMetadata (SettableBlobMeta meta) {
+        this.meta = meta;
+    }
+
+    @Override
+    public String toString() {
+        return _path+":"+(_isTmp ? "tmp": BlobStoreFile.BLOBSTORE_DATA_FILE)+":"+_key;
+    }
+
+    @Override
+    public long getFileLength() {
+        return _path.length();
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/blobstore/NimbusBlobStore.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/blobstore/NimbusBlobStore.java b/storm-core/src/jvm/backtype/storm/blobstore/NimbusBlobStore.java
new file mode 100644
index 0000000..bf084bb
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/blobstore/NimbusBlobStore.java
@@ -0,0 +1,412 @@
+/**
+ * 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() throws TException {
+            if (!eof) {
+                offset = 0;
+                synchronized(client) {
+                    listBlobs = client.getClient().listBlobs(listBlobs.get_session());
+                }
+                if (listBlobs.get_keys_size() == 0) {
+                    eof = true;
+                }
+            }
+        }
+
+        @Override
+        public synchronized boolean hasNext() {
+            try {
+                if (isCacheEmpty()) {
+                    readMore();
+                }
+            } catch (TException e) {
+                throw new RuntimeException(e);
+            }
+            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 {
+            try {
+                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;
+            } catch(TException exp) {
+                throw new IOException(exp);
+            }
+        }
+
+        @Override
+        public synchronized int read(byte[] b, int off, int len) throws IOException {
+            try {
+                if (isEmpty()) {
+                    readMore();
+                    if (eof) {
+                        return -1;
+                    }
+                }
+                int length = Math.min(len, available());
+                System.arraycopy(buffer, offset, b, off, length);
+                offset += length;
+                return length;
+            } catch(TException exp) {
+                throw new IOException(exp);
+            }
+        }
+
+        private boolean isEmpty() {
+            return buffer == null || offset >= end;
+        }
+
+        private void readMore() throws TException {
+            if (!eof) {
+                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;
+                }
+            }
+        }
+
+        @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() {
+        shutdown();
+    }
+
+    @Override
+    public void shutdown() {
+        if (client != null) {
+            client.close();
+            client = null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/cluster/ClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/cluster/ClusterState.java b/storm-core/src/jvm/backtype/storm/cluster/ClusterState.java
index 1bcc645..1960371 100644
--- a/storm-core/src/jvm/backtype/storm/cluster/ClusterState.java
+++ b/storm-core/src/jvm/backtype/storm/cluster/ClusterState.java
@@ -205,4 +205,13 @@ public interface ClusterState {
      * @param path The path to synchronize.
      */
     void sync_path(String path);
+
+    /**
+     * Allows us to delete the znodes within /storm/blobstore/key_name
+     * whose znodes start with the corresponding nimbusHostPortInfo
+     * @param path /storm/blobstore/key_name
+     * @param nimbusHostPortInfo Contains the host port information of
+     * a nimbus node.
+     */
+    void delete_node_blobstore(String path, String nimbusHostPortInfo);
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/codedistributor/ICodeDistributor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/codedistributor/ICodeDistributor.java b/storm-core/src/jvm/backtype/storm/codedistributor/ICodeDistributor.java
deleted file mode 100644
index c46688f..0000000
--- a/storm-core/src/jvm/backtype/storm/codedistributor/ICodeDistributor.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/**
- * 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.codedistributor;
-
-
-import java.io.File;
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Interface responsible to distribute code in the cluster.
- */
-public interface ICodeDistributor {
-    /**
-     * Prepare this code distributor.
-     * @param conf
-     */
-    void prepare(Map conf) throws Exception;
-
-    /**
-     * This API will perform the actual upload of the code to the distribution implementation.
-     * The API should return a Meta file which should have enough information for downloader
-     * so it can download the code e.g. for bittorrent it will be a torrent file, in case of something like HDFS or s3
-     * it might have the actual directory where all the code is put.
-     * @param dirPath directory where all the code to be distributed exists.
-     * @param topologyId the topologyId for which the meta file needs to be created.
-     * @return metaFile
-     */
-    File upload(String dirPath, String topologyId) throws Exception;
-
-    /**
-     * Given the topologyId and metafile, download the actual code and return the downloaded file's list.
-     * @param topologyid
-     * @param metafile
-     * @return
-     */
-    List<File> download(String topologyid, File metafile) throws Exception;
-
-    /**
-     * returns number of nodes to which the code is already replicated for the topology.
-     * @param topologyId
-     * @return
-     */
-    short getReplicationCount(String topologyId) throws Exception;
-
-    /**
-     * Performs the cleanup.
-     * @param topologyid
-     */
-    void cleanup(String topologyid) throws IOException;
-
-    /**
-     * Close this distributor.
-     * @param conf
-     */
-    void close(Map conf);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/codedistributor/LocalFileSystemCodeDistributor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/codedistributor/LocalFileSystemCodeDistributor.java b/storm-core/src/jvm/backtype/storm/codedistributor/LocalFileSystemCodeDistributor.java
deleted file mode 100644
index 76993e2..0000000
--- a/storm-core/src/jvm/backtype/storm/codedistributor/LocalFileSystemCodeDistributor.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/**
- * 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.codedistributor;
-
-import backtype.storm.nimbus.NimbusInfo;
-import backtype.storm.utils.ZookeeperAuthInfo;
-import com.google.common.collect.Lists;
-import org.apache.commons.io.FileUtils;
-import org.apache.curator.framework.CuratorFramework;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import static backtype.storm.Config.*;
-import static backtype.storm.utils.Utils.downloadFromHost;
-import static backtype.storm.utils.Utils.newCurator;
-
-
-public class LocalFileSystemCodeDistributor implements ICodeDistributor {
-    private static final Logger LOG = LoggerFactory.getLogger(LocalFileSystemCodeDistributor.class);
-    private CuratorFramework zkClient;
-    private Map conf;
-
-    @Override
-    public void prepare(Map conf) throws Exception {
-        this.conf = conf;
-        List<String> zkServers = (List<String>) conf.get(STORM_ZOOKEEPER_SERVERS);
-        int port = (Integer) conf.get(STORM_ZOOKEEPER_PORT);
-        ZookeeperAuthInfo zkAuthInfo = new ZookeeperAuthInfo(conf);
-        zkClient = newCurator(conf, zkServers, port, (String) conf.get(STORM_ZOOKEEPER_ROOT), zkAuthInfo);
-        zkClient.start();
-    }
-
-    @Override
-    public File upload(String dirPath, String topologyId) throws Exception {
-        ArrayList<File> files = new ArrayList<File>();
-        File destDir = new File(dirPath);
-        File[] localFiles = destDir.listFiles();
-
-        List<String> filePaths = new ArrayList<String>(3);
-        for (File file : localFiles) {
-            filePaths.add(file.getAbsolutePath());
-        }
-
-        File metaFile = new File(destDir, "storm-code-distributor.meta");
-        boolean isCreated = metaFile.createNewFile();
-        if (isCreated) {
-            FileUtils.writeLines(metaFile, filePaths);
-        } else {
-            LOG.warn("metafile " + metaFile.getAbsolutePath() + " already exists.");
-        }
-
-        LOG.info("Created meta file " + metaFile.getAbsolutePath() + " upload successful.");
-
-        return metaFile;
-    }
-
-    @Override
-    public List<File> download(String topologyid, File metafile) throws Exception {
-        List<String> hostInfos = zkClient.getChildren().forPath("/code-distributor/" + topologyid);
-        File destDir = metafile.getParentFile();
-        List<File> downloadedFiles = Lists.newArrayList();
-        for (String absolutePathOnRemote : FileUtils.readLines(metafile)) {
-
-            File localFile = new File(destDir, new File(absolutePathOnRemote).getName());
-
-            boolean isSuccess = false;
-            for (String hostAndPort : hostInfos) {
-                NimbusInfo nimbusInfo = NimbusInfo.parse(hostAndPort);
-                try {
-                    LOG.info("Attempting to download meta file {} from remote {}", absolutePathOnRemote, nimbusInfo.toHostPortString());
-                    downloadFromHost(conf, absolutePathOnRemote, localFile.getAbsolutePath(), nimbusInfo.getHost(), nimbusInfo.getPort());
-                    downloadedFiles.add(localFile);
-                    isSuccess = true;
-                    break;
-                } catch (Exception e) {
-                    LOG.error("download failed from {}:{}, will try another endpoint ", nimbusInfo.getHost(), nimbusInfo.getPort(), e);
-                }
-            }
-
-            if(!isSuccess) {
-                throw new RuntimeException("File " + absolutePathOnRemote +" could not be downloaded from any endpoint");
-            }
-        }
-
-        return downloadedFiles;
-    }
-
-    @Override
-    public short getReplicationCount(String topologyId) throws Exception {
-        return (short) zkClient.getChildren().forPath("/code-distributor/" + topologyId).size();
-    }
-
-    @Override
-    public void cleanup(String topologyid) throws IOException {
-        //no op.
-    }
-
-    @Override
-    public void close(Map conf) {
-       zkClient.close();
-    }
-}


[02/17] storm git commit: Blobstore API STORM- 876

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/py/storm/ttypes.py
----------------------------------------------------------------------
diff --git a/storm-core/src/py/storm/ttypes.py b/storm-core/src/py/storm/ttypes.py
index a730c13..98a7ba4 100644
--- a/storm-core/src/py/storm/ttypes.py
+++ b/storm-core/src/py/storm/ttypes.py
@@ -62,6 +62,20 @@ class TopologyInitialStatus:
     "INACTIVE": 2,
   }
 
+class AccessControlType:
+  OTHER = 1
+  USER = 2
+
+  _VALUES_TO_NAMES = {
+    1: "OTHER",
+    2: "USER",
+  }
+
+  _NAMES_TO_VALUES = {
+    "OTHER": 1,
+    "USER": 2,
+  }
+
 class TopologyStatus:
   ACTIVE = 1
   INACTIVE = 2
@@ -1802,6 +1816,146 @@ class InvalidTopologyException(TException):
   def __ne__(self, other):
     return not (self == other)
 
+class KeyNotFoundException(TException):
+  """
+  Attributes:
+   - msg
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'msg', None, None, ), # 1
+  )
+
+  def __init__(self, msg=None,):
+    self.msg = msg
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.msg = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('KeyNotFoundException')
+    if self.msg is not None:
+      oprot.writeFieldBegin('msg', TType.STRING, 1)
+      oprot.writeString(self.msg.encode('utf-8'))
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.msg is None:
+      raise TProtocol.TProtocolException(message='Required field msg is unset!')
+    return
+
+
+  def __str__(self):
+    return repr(self)
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.msg)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class KeyAlreadyExistsException(TException):
+  """
+  Attributes:
+   - msg
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'msg', None, None, ), # 1
+  )
+
+  def __init__(self, msg=None,):
+    self.msg = msg
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.msg = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('KeyAlreadyExistsException')
+    if self.msg is not None:
+      oprot.writeFieldBegin('msg', TType.STRING, 1)
+      oprot.writeString(self.msg.encode('utf-8'))
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.msg is None:
+      raise TProtocol.TProtocolException(message='Required field msg is unset!')
+    return
+
+
+  def __str__(self):
+    return repr(self)
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.msg)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class TopologySummary:
   """
   Attributes:
@@ -7110,6 +7264,458 @@ class SubmitOptions:
   def __ne__(self, other):
     return not (self == other)
 
+class AccessControl:
+  """
+  Attributes:
+   - type
+   - name
+   - access
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.I32, 'type', None, None, ), # 1
+    (2, TType.STRING, 'name', None, None, ), # 2
+    (3, TType.I32, 'access', None, None, ), # 3
+  )
+
+  def __init__(self, type=None, name=None, access=None,):
+    self.type = type
+    self.name = name
+    self.access = access
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.I32:
+          self.type = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.name = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.I32:
+          self.access = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('AccessControl')
+    if self.type is not None:
+      oprot.writeFieldBegin('type', TType.I32, 1)
+      oprot.writeI32(self.type)
+      oprot.writeFieldEnd()
+    if self.name is not None:
+      oprot.writeFieldBegin('name', TType.STRING, 2)
+      oprot.writeString(self.name.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.access is not None:
+      oprot.writeFieldBegin('access', TType.I32, 3)
+      oprot.writeI32(self.access)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.type is None:
+      raise TProtocol.TProtocolException(message='Required field type is unset!')
+    if self.access is None:
+      raise TProtocol.TProtocolException(message='Required field access is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.type)
+    value = (value * 31) ^ hash(self.name)
+    value = (value * 31) ^ hash(self.access)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class SettableBlobMeta:
+  """
+  Attributes:
+   - acl
+   - replication_factor
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'acl', (TType.STRUCT,(AccessControl, AccessControl.thrift_spec)), None, ), # 1
+    (2, TType.I32, 'replication_factor', None, None, ), # 2
+  )
+
+  def __init__(self, acl=None, replication_factor=None,):
+    self.acl = acl
+    self.replication_factor = replication_factor
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.LIST:
+          self.acl = []
+          (_etype440, _size437) = iprot.readListBegin()
+          for _i441 in xrange(_size437):
+            _elem442 = AccessControl()
+            _elem442.read(iprot)
+            self.acl.append(_elem442)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I32:
+          self.replication_factor = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('SettableBlobMeta')
+    if self.acl is not None:
+      oprot.writeFieldBegin('acl', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRUCT, len(self.acl))
+      for iter443 in self.acl:
+        iter443.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.replication_factor is not None:
+      oprot.writeFieldBegin('replication_factor', TType.I32, 2)
+      oprot.writeI32(self.replication_factor)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.acl is None:
+      raise TProtocol.TProtocolException(message='Required field acl is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.acl)
+    value = (value * 31) ^ hash(self.replication_factor)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class ReadableBlobMeta:
+  """
+  Attributes:
+   - settable
+   - version
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'settable', (SettableBlobMeta, SettableBlobMeta.thrift_spec), None, ), # 1
+    (2, TType.I64, 'version', None, None, ), # 2
+  )
+
+  def __init__(self, settable=None, version=None,):
+    self.settable = settable
+    self.version = version
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.settable = SettableBlobMeta()
+          self.settable.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I64:
+          self.version = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('ReadableBlobMeta')
+    if self.settable is not None:
+      oprot.writeFieldBegin('settable', TType.STRUCT, 1)
+      self.settable.write(oprot)
+      oprot.writeFieldEnd()
+    if self.version is not None:
+      oprot.writeFieldBegin('version', TType.I64, 2)
+      oprot.writeI64(self.version)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.settable is None:
+      raise TProtocol.TProtocolException(message='Required field settable is unset!')
+    if self.version is None:
+      raise TProtocol.TProtocolException(message='Required field version is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.settable)
+    value = (value * 31) ^ hash(self.version)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class ListBlobsResult:
+  """
+  Attributes:
+   - keys
+   - session
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'keys', (TType.STRING,None), None, ), # 1
+    (2, TType.STRING, 'session', None, None, ), # 2
+  )
+
+  def __init__(self, keys=None, session=None,):
+    self.keys = keys
+    self.session = session
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.LIST:
+          self.keys = []
+          (_etype447, _size444) = iprot.readListBegin()
+          for _i448 in xrange(_size444):
+            _elem449 = iprot.readString().decode('utf-8')
+            self.keys.append(_elem449)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.session = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('ListBlobsResult')
+    if self.keys is not None:
+      oprot.writeFieldBegin('keys', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRING, len(self.keys))
+      for iter450 in self.keys:
+        oprot.writeString(iter450.encode('utf-8'))
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.session is not None:
+      oprot.writeFieldBegin('session', TType.STRING, 2)
+      oprot.writeString(self.session.encode('utf-8'))
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.keys is None:
+      raise TProtocol.TProtocolException(message='Required field keys is unset!')
+    if self.session is None:
+      raise TProtocol.TProtocolException(message='Required field session is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.keys)
+    value = (value * 31) ^ hash(self.session)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class BeginDownloadResult:
+  """
+  Attributes:
+   - version
+   - session
+   - data_size
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.I64, 'version', None, None, ), # 1
+    (2, TType.STRING, 'session', None, None, ), # 2
+    (3, TType.I64, 'data_size', None, None, ), # 3
+  )
+
+  def __init__(self, version=None, session=None, data_size=None,):
+    self.version = version
+    self.session = session
+    self.data_size = data_size
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.I64:
+          self.version = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.session = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.I64:
+          self.data_size = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('BeginDownloadResult')
+    if self.version is not None:
+      oprot.writeFieldBegin('version', TType.I64, 1)
+      oprot.writeI64(self.version)
+      oprot.writeFieldEnd()
+    if self.session is not None:
+      oprot.writeFieldBegin('session', TType.STRING, 2)
+      oprot.writeString(self.session.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.data_size is not None:
+      oprot.writeFieldBegin('data_size', TType.I64, 3)
+      oprot.writeI64(self.data_size)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.version is None:
+      raise TProtocol.TProtocolException(message='Required field version is unset!')
+    if self.session is None:
+      raise TProtocol.TProtocolException(message='Required field session is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.version)
+    value = (value * 31) ^ hash(self.session)
+    value = (value * 31) ^ hash(self.data_size)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class SupervisorInfo:
   """
   Attributes:
@@ -7175,31 +7781,31 @@ class SupervisorInfo:
       elif fid == 4:
         if ftype == TType.LIST:
           self.used_ports = []
-          (_etype440, _size437) = iprot.readListBegin()
-          for _i441 in xrange(_size437):
-            _elem442 = iprot.readI64()
-            self.used_ports.append(_elem442)
+          (_etype454, _size451) = iprot.readListBegin()
+          for _i455 in xrange(_size451):
+            _elem456 = iprot.readI64()
+            self.used_ports.append(_elem456)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.LIST:
           self.meta = []
-          (_etype446, _size443) = iprot.readListBegin()
-          for _i447 in xrange(_size443):
-            _elem448 = iprot.readI64()
-            self.meta.append(_elem448)
+          (_etype460, _size457) = iprot.readListBegin()
+          for _i461 in xrange(_size457):
+            _elem462 = iprot.readI64()
+            self.meta.append(_elem462)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 6:
         if ftype == TType.MAP:
           self.scheduler_meta = {}
-          (_ktype450, _vtype451, _size449 ) = iprot.readMapBegin()
-          for _i453 in xrange(_size449):
-            _key454 = iprot.readString().decode('utf-8')
-            _val455 = iprot.readString().decode('utf-8')
-            self.scheduler_meta[_key454] = _val455
+          (_ktype464, _vtype465, _size463 ) = iprot.readMapBegin()
+          for _i467 in xrange(_size463):
+            _key468 = iprot.readString().decode('utf-8')
+            _val469 = iprot.readString().decode('utf-8')
+            self.scheduler_meta[_key468] = _val469
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -7216,11 +7822,11 @@ class SupervisorInfo:
       elif fid == 9:
         if ftype == TType.MAP:
           self.resources_map = {}
-          (_ktype457, _vtype458, _size456 ) = iprot.readMapBegin()
-          for _i460 in xrange(_size456):
-            _key461 = iprot.readString().decode('utf-8')
-            _val462 = iprot.readDouble()
-            self.resources_map[_key461] = _val462
+          (_ktype471, _vtype472, _size470 ) = iprot.readMapBegin()
+          for _i474 in xrange(_size470):
+            _key475 = iprot.readString().decode('utf-8')
+            _val476 = iprot.readDouble()
+            self.resources_map[_key475] = _val476
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -7249,23 +7855,23 @@ class SupervisorInfo:
     if self.used_ports is not None:
       oprot.writeFieldBegin('used_ports', TType.LIST, 4)
       oprot.writeListBegin(TType.I64, len(self.used_ports))
-      for iter463 in self.used_ports:
-        oprot.writeI64(iter463)
+      for iter477 in self.used_ports:
+        oprot.writeI64(iter477)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.meta is not None:
       oprot.writeFieldBegin('meta', TType.LIST, 5)
       oprot.writeListBegin(TType.I64, len(self.meta))
-      for iter464 in self.meta:
-        oprot.writeI64(iter464)
+      for iter478 in self.meta:
+        oprot.writeI64(iter478)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.scheduler_meta is not None:
       oprot.writeFieldBegin('scheduler_meta', TType.MAP, 6)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.scheduler_meta))
-      for kiter465,viter466 in self.scheduler_meta.items():
-        oprot.writeString(kiter465.encode('utf-8'))
-        oprot.writeString(viter466.encode('utf-8'))
+      for kiter479,viter480 in self.scheduler_meta.items():
+        oprot.writeString(kiter479.encode('utf-8'))
+        oprot.writeString(viter480.encode('utf-8'))
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.uptime_secs is not None:
@@ -7279,9 +7885,9 @@ class SupervisorInfo:
     if self.resources_map is not None:
       oprot.writeFieldBegin('resources_map', TType.MAP, 9)
       oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(self.resources_map))
-      for kiter467,viter468 in self.resources_map.items():
-        oprot.writeString(kiter467.encode('utf-8'))
-        oprot.writeDouble(viter468)
+      for kiter481,viter482 in self.resources_map.items():
+        oprot.writeString(kiter481.encode('utf-8'))
+        oprot.writeDouble(viter482)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7353,10 +7959,10 @@ class NodeInfo:
       elif fid == 2:
         if ftype == TType.SET:
           self.port = set()
-          (_etype472, _size469) = iprot.readSetBegin()
-          for _i473 in xrange(_size469):
-            _elem474 = iprot.readI64()
-            self.port.add(_elem474)
+          (_etype486, _size483) = iprot.readSetBegin()
+          for _i487 in xrange(_size483):
+            _elem488 = iprot.readI64()
+            self.port.add(_elem488)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
@@ -7377,8 +7983,8 @@ class NodeInfo:
     if self.port is not None:
       oprot.writeFieldBegin('port', TType.SET, 2)
       oprot.writeSetBegin(TType.I64, len(self.port))
-      for iter475 in self.port:
-        oprot.writeI64(iter475)
+      for iter489 in self.port:
+        oprot.writeI64(iter489)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7559,57 +8165,57 @@ class Assignment:
       elif fid == 2:
         if ftype == TType.MAP:
           self.node_host = {}
-          (_ktype477, _vtype478, _size476 ) = iprot.readMapBegin()
-          for _i480 in xrange(_size476):
-            _key481 = iprot.readString().decode('utf-8')
-            _val482 = iprot.readString().decode('utf-8')
-            self.node_host[_key481] = _val482
+          (_ktype491, _vtype492, _size490 ) = iprot.readMapBegin()
+          for _i494 in xrange(_size490):
+            _key495 = iprot.readString().decode('utf-8')
+            _val496 = iprot.readString().decode('utf-8')
+            self.node_host[_key495] = _val496
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.MAP:
           self.executor_node_port = {}
-          (_ktype484, _vtype485, _size483 ) = iprot.readMapBegin()
-          for _i487 in xrange(_size483):
-            _key488 = []
-            (_etype493, _size490) = iprot.readListBegin()
-            for _i494 in xrange(_size490):
-              _elem495 = iprot.readI64()
-              _key488.append(_elem495)
+          (_ktype498, _vtype499, _size497 ) = iprot.readMapBegin()
+          for _i501 in xrange(_size497):
+            _key502 = []
+            (_etype507, _size504) = iprot.readListBegin()
+            for _i508 in xrange(_size504):
+              _elem509 = iprot.readI64()
+              _key502.append(_elem509)
             iprot.readListEnd()
-            _val489 = NodeInfo()
-            _val489.read(iprot)
-            self.executor_node_port[_key488] = _val489
+            _val503 = NodeInfo()
+            _val503.read(iprot)
+            self.executor_node_port[_key502] = _val503
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.MAP:
           self.executor_start_time_secs = {}
-          (_ktype497, _vtype498, _size496 ) = iprot.readMapBegin()
-          for _i500 in xrange(_size496):
-            _key501 = []
-            (_etype506, _size503) = iprot.readListBegin()
-            for _i507 in xrange(_size503):
-              _elem508 = iprot.readI64()
-              _key501.append(_elem508)
+          (_ktype511, _vtype512, _size510 ) = iprot.readMapBegin()
+          for _i514 in xrange(_size510):
+            _key515 = []
+            (_etype520, _size517) = iprot.readListBegin()
+            for _i521 in xrange(_size517):
+              _elem522 = iprot.readI64()
+              _key515.append(_elem522)
             iprot.readListEnd()
-            _val502 = iprot.readI64()
-            self.executor_start_time_secs[_key501] = _val502
+            _val516 = iprot.readI64()
+            self.executor_start_time_secs[_key515] = _val516
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.MAP:
           self.worker_resources = {}
-          (_ktype510, _vtype511, _size509 ) = iprot.readMapBegin()
-          for _i513 in xrange(_size509):
-            _key514 = NodeInfo()
-            _key514.read(iprot)
-            _val515 = WorkerResources()
-            _val515.read(iprot)
-            self.worker_resources[_key514] = _val515
+          (_ktype524, _vtype525, _size523 ) = iprot.readMapBegin()
+          for _i527 in xrange(_size523):
+            _key528 = NodeInfo()
+            _key528.read(iprot)
+            _val529 = WorkerResources()
+            _val529.read(iprot)
+            self.worker_resources[_key528] = _val529
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -7630,39 +8236,39 @@ class Assignment:
     if self.node_host is not None:
       oprot.writeFieldBegin('node_host', TType.MAP, 2)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.node_host))
-      for kiter516,viter517 in self.node_host.items():
-        oprot.writeString(kiter516.encode('utf-8'))
-        oprot.writeString(viter517.encode('utf-8'))
+      for kiter530,viter531 in self.node_host.items():
+        oprot.writeString(kiter530.encode('utf-8'))
+        oprot.writeString(viter531.encode('utf-8'))
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.executor_node_port is not None:
       oprot.writeFieldBegin('executor_node_port', TType.MAP, 3)
       oprot.writeMapBegin(TType.LIST, TType.STRUCT, len(self.executor_node_port))
-      for kiter518,viter519 in self.executor_node_port.items():
-        oprot.writeListBegin(TType.I64, len(kiter518))
-        for iter520 in kiter518:
-          oprot.writeI64(iter520)
+      for kiter532,viter533 in self.executor_node_port.items():
+        oprot.writeListBegin(TType.I64, len(kiter532))
+        for iter534 in kiter532:
+          oprot.writeI64(iter534)
         oprot.writeListEnd()
-        viter519.write(oprot)
+        viter533.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.executor_start_time_secs is not None:
       oprot.writeFieldBegin('executor_start_time_secs', TType.MAP, 4)
       oprot.writeMapBegin(TType.LIST, TType.I64, len(self.executor_start_time_secs))
-      for kiter521,viter522 in self.executor_start_time_secs.items():
-        oprot.writeListBegin(TType.I64, len(kiter521))
-        for iter523 in kiter521:
-          oprot.writeI64(iter523)
+      for kiter535,viter536 in self.executor_start_time_secs.items():
+        oprot.writeListBegin(TType.I64, len(kiter535))
+        for iter537 in kiter535:
+          oprot.writeI64(iter537)
         oprot.writeListEnd()
-        oprot.writeI64(viter522)
+        oprot.writeI64(viter536)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.worker_resources is not None:
       oprot.writeFieldBegin('worker_resources', TType.MAP, 5)
       oprot.writeMapBegin(TType.STRUCT, TType.STRUCT, len(self.worker_resources))
-      for kiter524,viter525 in self.worker_resources.items():
-        kiter524.write(oprot)
-        viter525.write(oprot)
+      for kiter538,viter539 in self.worker_resources.items():
+        kiter538.write(oprot)
+        viter539.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7839,11 +8445,11 @@ class StormBase:
       elif fid == 4:
         if ftype == TType.MAP:
           self.component_executors = {}
-          (_ktype527, _vtype528, _size526 ) = iprot.readMapBegin()
-          for _i530 in xrange(_size526):
-            _key531 = iprot.readString().decode('utf-8')
-            _val532 = iprot.readI32()
-            self.component_executors[_key531] = _val532
+          (_ktype541, _vtype542, _size540 ) = iprot.readMapBegin()
+          for _i544 in xrange(_size540):
+            _key545 = iprot.readString().decode('utf-8')
+            _val546 = iprot.readI32()
+            self.component_executors[_key545] = _val546
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -7871,12 +8477,12 @@ class StormBase:
       elif fid == 9:
         if ftype == TType.MAP:
           self.component_debug = {}
-          (_ktype534, _vtype535, _size533 ) = iprot.readMapBegin()
-          for _i537 in xrange(_size533):
-            _key538 = iprot.readString().decode('utf-8')
-            _val539 = DebugOptions()
-            _val539.read(iprot)
-            self.component_debug[_key538] = _val539
+          (_ktype548, _vtype549, _size547 ) = iprot.readMapBegin()
+          for _i551 in xrange(_size547):
+            _key552 = iprot.readString().decode('utf-8')
+            _val553 = DebugOptions()
+            _val553.read(iprot)
+            self.component_debug[_key552] = _val553
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -7905,9 +8511,9 @@ class StormBase:
     if self.component_executors is not None:
       oprot.writeFieldBegin('component_executors', TType.MAP, 4)
       oprot.writeMapBegin(TType.STRING, TType.I32, len(self.component_executors))
-      for kiter540,viter541 in self.component_executors.items():
-        oprot.writeString(kiter540.encode('utf-8'))
-        oprot.writeI32(viter541)
+      for kiter554,viter555 in self.component_executors.items():
+        oprot.writeString(kiter554.encode('utf-8'))
+        oprot.writeI32(viter555)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.launch_time_secs is not None:
@@ -7929,9 +8535,9 @@ class StormBase:
     if self.component_debug is not None:
       oprot.writeFieldBegin('component_debug', TType.MAP, 9)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.component_debug))
-      for kiter542,viter543 in self.component_debug.items():
-        oprot.writeString(kiter542.encode('utf-8'))
-        viter543.write(oprot)
+      for kiter556,viter557 in self.component_debug.items():
+        oprot.writeString(kiter556.encode('utf-8'))
+        viter557.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -8011,13 +8617,13 @@ class ClusterWorkerHeartbeat:
       elif fid == 2:
         if ftype == TType.MAP:
           self.executor_stats = {}
-          (_ktype545, _vtype546, _size544 ) = iprot.readMapBegin()
-          for _i548 in xrange(_size544):
-            _key549 = ExecutorInfo()
-            _key549.read(iprot)
-            _val550 = ExecutorStats()
-            _val550.read(iprot)
-            self.executor_stats[_key549] = _val550
+          (_ktype559, _vtype560, _size558 ) = iprot.readMapBegin()
+          for _i562 in xrange(_size558):
+            _key563 = ExecutorInfo()
+            _key563.read(iprot)
+            _val564 = ExecutorStats()
+            _val564.read(iprot)
+            self.executor_stats[_key563] = _val564
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -8048,9 +8654,9 @@ class ClusterWorkerHeartbeat:
     if self.executor_stats is not None:
       oprot.writeFieldBegin('executor_stats', TType.MAP, 2)
       oprot.writeMapBegin(TType.STRUCT, TType.STRUCT, len(self.executor_stats))
-      for kiter551,viter552 in self.executor_stats.items():
-        kiter551.write(oprot)
-        viter552.write(oprot)
+      for kiter565,viter566 in self.executor_stats.items():
+        kiter565.write(oprot)
+        viter566.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.time_secs is not None:
@@ -8203,12 +8809,12 @@ class LocalStateData:
       if fid == 1:
         if ftype == TType.MAP:
           self.serialized_parts = {}
-          (_ktype554, _vtype555, _size553 ) = iprot.readMapBegin()
-          for _i557 in xrange(_size553):
-            _key558 = iprot.readString().decode('utf-8')
-            _val559 = ThriftSerializedObject()
-            _val559.read(iprot)
-            self.serialized_parts[_key558] = _val559
+          (_ktype568, _vtype569, _size567 ) = iprot.readMapBegin()
+          for _i571 in xrange(_size567):
+            _key572 = iprot.readString().decode('utf-8')
+            _val573 = ThriftSerializedObject()
+            _val573.read(iprot)
+            self.serialized_parts[_key572] = _val573
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -8225,9 +8831,9 @@ class LocalStateData:
     if self.serialized_parts is not None:
       oprot.writeFieldBegin('serialized_parts', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.serialized_parts))
-      for kiter560,viter561 in self.serialized_parts.items():
-        oprot.writeString(kiter560.encode('utf-8'))
-        viter561.write(oprot)
+      for kiter574,viter575 in self.serialized_parts.items():
+        oprot.writeString(kiter574.encode('utf-8'))
+        viter575.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -8292,11 +8898,11 @@ class LocalAssignment:
       elif fid == 2:
         if ftype == TType.LIST:
           self.executors = []
-          (_etype565, _size562) = iprot.readListBegin()
-          for _i566 in xrange(_size562):
-            _elem567 = ExecutorInfo()
-            _elem567.read(iprot)
-            self.executors.append(_elem567)
+          (_etype579, _size576) = iprot.readListBegin()
+          for _i580 in xrange(_size576):
+            _elem581 = ExecutorInfo()
+            _elem581.read(iprot)
+            self.executors.append(_elem581)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -8323,8 +8929,8 @@ class LocalAssignment:
     if self.executors is not None:
       oprot.writeFieldBegin('executors', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.executors))
-      for iter568 in self.executors:
-        iter568.write(oprot)
+      for iter582 in self.executors:
+        iter582.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.resources is not None:
@@ -8453,11 +9059,11 @@ class LSApprovedWorkers:
       if fid == 1:
         if ftype == TType.MAP:
           self.approved_workers = {}
-          (_ktype570, _vtype571, _size569 ) = iprot.readMapBegin()
-          for _i573 in xrange(_size569):
-            _key574 = iprot.readString().decode('utf-8')
-            _val575 = iprot.readI32()
-            self.approved_workers[_key574] = _val575
+          (_ktype584, _vtype585, _size583 ) = iprot.readMapBegin()
+          for _i587 in xrange(_size583):
+            _key588 = iprot.readString().decode('utf-8')
+            _val589 = iprot.readI32()
+            self.approved_workers[_key588] = _val589
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -8474,9 +9080,9 @@ class LSApprovedWorkers:
     if self.approved_workers is not None:
       oprot.writeFieldBegin('approved_workers', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.I32, len(self.approved_workers))
-      for kiter576,viter577 in self.approved_workers.items():
-        oprot.writeString(kiter576.encode('utf-8'))
-        oprot.writeI32(viter577)
+      for kiter590,viter591 in self.approved_workers.items():
+        oprot.writeString(kiter590.encode('utf-8'))
+        oprot.writeI32(viter591)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -8530,12 +9136,12 @@ class LSSupervisorAssignments:
       if fid == 1:
         if ftype == TType.MAP:
           self.assignments = {}
-          (_ktype579, _vtype580, _size578 ) = iprot.readMapBegin()
-          for _i582 in xrange(_size578):
-            _key583 = iprot.readI32()
-            _val584 = LocalAssignment()
-            _val584.read(iprot)
-            self.assignments[_key583] = _val584
+          (_ktype593, _vtype594, _size592 ) = iprot.readMapBegin()
+          for _i596 in xrange(_size592):
+            _key597 = iprot.readI32()
+            _val598 = LocalAssignment()
+            _val598.read(iprot)
+            self.assignments[_key597] = _val598
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -8552,9 +9158,9 @@ class LSSupervisorAssignments:
     if self.assignments is not None:
       oprot.writeFieldBegin('assignments', TType.MAP, 1)
       oprot.writeMapBegin(TType.I32, TType.STRUCT, len(self.assignments))
-      for kiter585,viter586 in self.assignments.items():
-        oprot.writeI32(kiter585)
-        viter586.write(oprot)
+      for kiter599,viter600 in self.assignments.items():
+        oprot.writeI32(kiter599)
+        viter600.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -8627,11 +9233,11 @@ class LSWorkerHeartbeat:
       elif fid == 3:
         if ftype == TType.LIST:
           self.executors = []
-          (_etype590, _size587) = iprot.readListBegin()
-          for _i591 in xrange(_size587):
-            _elem592 = ExecutorInfo()
-            _elem592.read(iprot)
-            self.executors.append(_elem592)
+          (_etype604, _size601) = iprot.readListBegin()
+          for _i605 in xrange(_size601):
+            _elem606 = ExecutorInfo()
+            _elem606.read(iprot)
+            self.executors.append(_elem606)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -8661,8 +9267,8 @@ class LSWorkerHeartbeat:
     if self.executors is not None:
       oprot.writeFieldBegin('executors', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.executors))
-      for iter593 in self.executors:
-        iter593.write(oprot)
+      for iter607 in self.executors:
+        iter607.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.port is not None:
@@ -8748,20 +9354,20 @@ class LSTopoHistory:
       elif fid == 3:
         if ftype == TType.LIST:
           self.users = []
-          (_etype597, _size594) = iprot.readListBegin()
-          for _i598 in xrange(_size594):
-            _elem599 = iprot.readString().decode('utf-8')
-            self.users.append(_elem599)
+          (_etype611, _size608) = iprot.readListBegin()
+          for _i612 in xrange(_size608):
+            _elem613 = iprot.readString().decode('utf-8')
+            self.users.append(_elem613)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.groups = []
-          (_etype603, _size600) = iprot.readListBegin()
-          for _i604 in xrange(_size600):
-            _elem605 = iprot.readString().decode('utf-8')
-            self.groups.append(_elem605)
+          (_etype617, _size614) = iprot.readListBegin()
+          for _i618 in xrange(_size614):
+            _elem619 = iprot.readString().decode('utf-8')
+            self.groups.append(_elem619)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -8786,15 +9392,15 @@ class LSTopoHistory:
     if self.users is not None:
       oprot.writeFieldBegin('users', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.users))
-      for iter606 in self.users:
-        oprot.writeString(iter606.encode('utf-8'))
+      for iter620 in self.users:
+        oprot.writeString(iter620.encode('utf-8'))
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.groups is not None:
       oprot.writeFieldBegin('groups', TType.LIST, 4)
       oprot.writeListBegin(TType.STRING, len(self.groups))
-      for iter607 in self.groups:
-        oprot.writeString(iter607.encode('utf-8'))
+      for iter621 in self.groups:
+        oprot.writeString(iter621.encode('utf-8'))
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -8857,11 +9463,11 @@ class LSTopoHistoryList:
       if fid == 1:
         if ftype == TType.LIST:
           self.topo_history = []
-          (_etype611, _size608) = iprot.readListBegin()
-          for _i612 in xrange(_size608):
-            _elem613 = LSTopoHistory()
-            _elem613.read(iprot)
-            self.topo_history.append(_elem613)
+          (_etype625, _size622) = iprot.readListBegin()
+          for _i626 in xrange(_size622):
+            _elem627 = LSTopoHistory()
+            _elem627.read(iprot)
+            self.topo_history.append(_elem627)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -8878,8 +9484,8 @@ class LSTopoHistoryList:
     if self.topo_history is not None:
       oprot.writeFieldBegin('topo_history', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.topo_history))
-      for iter614 in self.topo_history:
-        iter614.write(oprot)
+      for iter628 in self.topo_history:
+        iter628.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -9214,12 +9820,12 @@ class LogConfig:
       if fid == 2:
         if ftype == TType.MAP:
           self.named_logger_level = {}
-          (_ktype616, _vtype617, _size615 ) = iprot.readMapBegin()
-          for _i619 in xrange(_size615):
-            _key620 = iprot.readString().decode('utf-8')
-            _val621 = LogLevel()
-            _val621.read(iprot)
-            self.named_logger_level[_key620] = _val621
+          (_ktype630, _vtype631, _size629 ) = iprot.readMapBegin()
+          for _i633 in xrange(_size629):
+            _key634 = iprot.readString().decode('utf-8')
+            _val635 = LogLevel()
+            _val635.read(iprot)
+            self.named_logger_level[_key634] = _val635
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -9236,9 +9842,9 @@ class LogConfig:
     if self.named_logger_level is not None:
       oprot.writeFieldBegin('named_logger_level', TType.MAP, 2)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.named_logger_level))
-      for kiter622,viter623 in self.named_logger_level.items():
-        oprot.writeString(kiter622.encode('utf-8'))
-        viter623.write(oprot)
+      for kiter636,viter637 in self.named_logger_level.items():
+        oprot.writeString(kiter636.encode('utf-8'))
+        viter637.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -9290,10 +9896,10 @@ class TopologyHistoryInfo:
       if fid == 1:
         if ftype == TType.LIST:
           self.topo_ids = []
-          (_etype627, _size624) = iprot.readListBegin()
-          for _i628 in xrange(_size624):
-            _elem629 = iprot.readString().decode('utf-8')
-            self.topo_ids.append(_elem629)
+          (_etype641, _size638) = iprot.readListBegin()
+          for _i642 in xrange(_size638):
+            _elem643 = iprot.readString().decode('utf-8')
+            self.topo_ids.append(_elem643)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9310,8 +9916,8 @@ class TopologyHistoryInfo:
     if self.topo_ids is not None:
       oprot.writeFieldBegin('topo_ids', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.topo_ids))
-      for iter630 in self.topo_ids:
-        oprot.writeString(iter630.encode('utf-8'))
+      for iter644 in self.topo_ids:
+        oprot.writeString(iter644.encode('utf-8'))
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -9961,11 +10567,11 @@ class HBRecords:
       if fid == 1:
         if ftype == TType.LIST:
           self.pulses = []
-          (_etype627, _size624) = iprot.readListBegin()
-          for _i628 in xrange(_size624):
-            _elem629 = HBPulse()
-            _elem629.read(iprot)
-            self.pulses.append(_elem629)
+          (_etype648, _size645) = iprot.readListBegin()
+          for _i649 in xrange(_size645):
+            _elem650 = HBPulse()
+            _elem650.read(iprot)
+            self.pulses.append(_elem650)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9982,8 +10588,8 @@ class HBRecords:
     if self.pulses is not None:
       oprot.writeFieldBegin('pulses', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.pulses))
-      for iter630 in self.pulses:
-        iter630.write(oprot)
+      for iter651 in self.pulses:
+        iter651.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -10035,10 +10641,10 @@ class HBNodes:
       if fid == 1:
         if ftype == TType.LIST:
           self.pulseIds = []
-          (_etype634, _size631) = iprot.readListBegin()
-          for _i635 in xrange(_size631):
-            _elem636 = iprot.readString().decode('utf-8')
-            self.pulseIds.append(_elem636)
+          (_etype655, _size652) = iprot.readListBegin()
+          for _i656 in xrange(_size652):
+            _elem657 = iprot.readString().decode('utf-8')
+            self.pulseIds.append(_elem657)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10055,8 +10661,8 @@ class HBNodes:
     if self.pulseIds is not None:
       oprot.writeFieldBegin('pulseIds', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.pulseIds))
-      for iter637 in self.pulseIds:
-        oprot.writeString(iter637.encode('utf-8'))
+      for iter658 in self.pulseIds:
+        oprot.writeString(iter658.encode('utf-8'))
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/storm.thrift
----------------------------------------------------------------------
diff --git a/storm-core/src/storm.thrift b/storm-core/src/storm.thrift
index 677de2b..08be005 100644
--- a/storm-core/src/storm.thrift
+++ b/storm-core/src/storm.thrift
@@ -136,6 +136,14 @@ exception InvalidTopologyException {
   1: required string msg;
 }
 
+exception KeyNotFoundException {
+  1: required string msg;
+}
+
+exception KeyAlreadyExistsException {
+  1: required string msg;
+}
+
 struct TopologySummary {
   1: required string id;
   2: required string name;
@@ -371,6 +379,42 @@ struct SubmitOptions {
   2: optional Credentials creds;
 }
 
+enum AccessControlType {
+  OTHER = 1,
+  USER = 2
+  //eventually ,GROUP=3
+}
+
+struct AccessControl {
+  1: required AccessControlType type;
+  2: optional string name; //Name of user or group in ACL
+  3: required i32 access; //bitmasks READ=0x1, WRITE=0x2, ADMIN=0x4
+}
+
+struct SettableBlobMeta {
+  1: required list<AccessControl> acl;
+  2: optional i32 replication_factor
+}
+
+struct ReadableBlobMeta {
+  1: required SettableBlobMeta settable;
+  //This is some indication of a version of a BLOB.  The only guarantee is
+  // if the data changed in the blob the version will be different.
+  2: required i64 version;
+}
+
+struct ListBlobsResult {
+  1: required list<string> keys;
+  2: required string session;
+}
+
+struct BeginDownloadResult {
+  //Same version as in ReadableBlobMeta
+  1: required i64 version;
+  2: required string session;
+  3: optional i64 data_size;
+}
+
 struct SupervisorInfo {
     1: required i64 time_secs;
     2: required string hostname;
@@ -565,6 +609,21 @@ service Nimbus {
 
   void uploadNewCredentials(1: string name, 2: Credentials creds) throws (1: NotAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze);
 
+  string beginCreateBlob(1: string key, 2: SettableBlobMeta meta) throws (1: AuthorizationException aze, 2: KeyAlreadyExistsException kae);
+  string beginUpdateBlob(1: string key) throws (1: AuthorizationException aze, 2: KeyNotFoundException knf);
+  void uploadBlobChunk(1: string session, 2: binary chunk) throws (1: AuthorizationException aze);
+  void finishBlobUpload(1: string session) throws (1: AuthorizationException aze);
+  void cancelBlobUpload(1: string session) throws (1: AuthorizationException aze);
+  ReadableBlobMeta getBlobMeta(1: string key) throws (1: AuthorizationException aze, 2: KeyNotFoundException knf);
+  void setBlobMeta(1: string key, 2: SettableBlobMeta meta) throws (1: AuthorizationException aze, 2: KeyNotFoundException knf);
+  BeginDownloadResult beginBlobDownload(1: string key) throws (1: AuthorizationException aze, 2: KeyNotFoundException knf);
+  binary downloadBlobChunk(1: string session) throws (1: AuthorizationException aze);
+  void deleteBlob(1: string key) throws (1: AuthorizationException aze, 2: KeyNotFoundException knf);
+  ListBlobsResult listBlobs(1: string session); //empty string "" means start at the beginning
+  i32 getBlobReplication(1: string key) throws (1: AuthorizationException aze, 2: KeyNotFoundException knf);
+  i32 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
+
   // need to add functions for asking about status of storms, what nodes they're running on, looking at task logs
 
   string beginFileUpload() throws (1: AuthorizationException aze);

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/test/clj/backtype/storm/cluster_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/cluster_test.clj b/storm-core/test/clj/backtype/storm/cluster_test.clj
index eea4637..82f305b 100644
--- a/storm-core/test/clj/backtype/storm/cluster_test.clj
+++ b/storm-core/test/clj/backtype/storm/cluster_test.clj
@@ -208,14 +208,15 @@
       (.set-credentials! state "storm1" {"b" "b"} {})
       (is (= {"b" "b"} (.credentials state "storm1" nil)))
 
-      (is (= [] (.code-distributor state nil)))
-      (.setup-code-distributor! state "storm1" nimbusInfo1)
-      (is (= ["storm1"] (.code-distributor state nil)))
-      (is (= [nimbusInfo1] (.code-distributor-info state "storm1")))
-      (.setup-code-distributor! state "storm1" nimbusInfo2)
-      (is (= #{nimbusInfo1 nimbusInfo2} (set (.code-distributor-info state "storm1"))))
-      (.remove-storm! state "storm1")
-      (is (= [] (.code-distributor state nil)))
+      (is (= [] (.blobstore-info state nil)))
+      (.setup-blobstore! state "key1" nimbusInfo1 "1")
+      (is (= ["key1"] (.blobstore-info state nil)))
+      (is (= [(str (.toHostPortString nimbusInfo1) "-1")] (.blobstore-info state "key1")))
+      (.setup-blobstore! state "key1" nimbusInfo2 "1")
+      (is (= #{(str (.toHostPortString nimbusInfo1) "-1")
+               (str (.toHostPortString nimbusInfo2) "-1")} (set (.blobstore-info state "key1"))))
+      (.remove-blobstore-key! state "key1")
+      (is (= [] (.blobstore-info state nil)))
 
       (is (= [] (.nimbuses state)))
       (.add-nimbus-host! state "nimbus1:port" nimbusSummary1)
@@ -266,8 +267,7 @@
     (let [state1 (mk-storm-state zk-port)
           state2 (mk-storm-state zk-port)
           supervisor-info1 (SupervisorInfo. 10 "hostname-1" "id1" [1 2] [] {} 1000 "0.9.2" nil)
-          supervisor-info2 (SupervisorInfo. 10 "hostname-2" "id2" [1 2] [] {} 1000 "0.9.2" nil)
-          ]
+          supervisor-info2 (SupervisorInfo. 10 "hostname-2" "id2" [1 2] [] {} 1000 "0.9.2" nil)]
       (is (= [] (.supervisors state1 nil)))
       (.supervisor-heartbeat! state2 "2" supervisor-info2)
       (.supervisor-heartbeat! state1 "1" supervisor-info1)

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/test/clj/backtype/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/nimbus_test.clj b/storm-core/test/clj/backtype/storm/nimbus_test.clj
index 53d4bb8..0847883 100644
--- a/storm-core/test/clj/backtype/storm/nimbus_test.clj
+++ b/storm-core/test/clj/backtype/storm/nimbus_test.clj
@@ -29,7 +29,7 @@
             LogConfig LogLevel LogLevelAction])
   (:import [java.util HashMap])
   (:import [java.io File])
-  (:import [backtype.storm.utils Time])
+  (:import [backtype.storm.utils Time Utils])
   (:import [org.apache.commons.io FileUtils])
   (:use [backtype.storm testing MockAutoCred util config log timer zookeeper])
   (:use [backtype.storm.daemon common])
@@ -939,41 +939,15 @@
          (bind storm-id1 (get-storm-id cluster-state "t1"))
          (bind storm-id2 (get-storm-id cluster-state "t2"))
          (.shutdown nimbus)
-         (rmr (master-stormdist-root conf storm-id1))
+         (let [blob-store (Utils/getNimbusBlobStore conf nil)]
+           (nimbus/blob-rm-topology-keys storm-id1 blob-store cluster-state)
+           (.shutdown blob-store))
          (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
          (is ( = #{storm-id2} (set (.active-storms cluster-state))))
          (.shutdown nimbus)
          (.disconnect cluster-state)
          )))))
 
-
-(deftest test-cleans-corrupt
-  (with-inprocess-zookeeper zk-port
-    (with-local-tmp [nimbus-dir]
-      (stubbing [zk-leader-elector (mock-leader-elector)]
-        (letlocals
-          (bind conf (merge (read-storm-config)
-                       {STORM-ZOOKEEPER-SERVERS ["localhost"]
-                        STORM-CLUSTER-MODE "local"
-                        STORM-ZOOKEEPER-PORT zk-port
-                        STORM-LOCAL-DIR nimbus-dir}))
-          (bind cluster-state (cluster/mk-storm-cluster-state conf))
-          (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
-          (bind topology (thrift/mk-topology
-                           {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
-                           {}))
-          (submit-local-topology nimbus "t1" {} topology)
-          (submit-local-topology nimbus "t2" {} topology)
-          (bind storm-id1 (get-storm-id cluster-state "t1"))
-          (bind storm-id2 (get-storm-id cluster-state "t2"))
-          (.shutdown nimbus)
-          (rmr (master-stormdist-root conf storm-id1))
-          (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
-          (is ( = #{storm-id2} (set (.active-storms cluster-state))))
-          (.shutdown nimbus)
-          (.disconnect cluster-state)
-          )))))
-
 ;(deftest test-no-overlapping-slots
 ;  ;; test that same node+port never appears across 2 assignments
 ;  )
@@ -1173,7 +1147,7 @@
                     nimbus/check-authorization!
                       [1 2 3] expected-name expected-conf expected-operation)
                   (verify-first-call-args-for-indices
-                    nimbus/try-read-storm-topology [0] expected-conf))))))))))
+                    nimbus/try-read-storm-topology [0] "fake-id"))))))))))
 
 (deftest test-nimbus-iface-getTopology-methods-throw-correctly
   (with-local-cluster [cluster]
@@ -1230,7 +1204,8 @@
                         :status {:type bogus-type}}
                 }
         ]
-      (stubbing [topology-bases bogus-bases]
+      (stubbing [topology-bases bogus-bases
+                 nimbus/get-blob-replication-count 1]
         (let [topos (.get_topologies (.getClusterInfo nimbus))]
           ; The number of topologies in the summary is correct.
           (is (= (count
@@ -1265,6 +1240,7 @@
           digest "storm:thisisapoorpassword"
           auth-conf {STORM-ZOOKEEPER-AUTH-SCHEME scheme
                      STORM-ZOOKEEPER-AUTH-PAYLOAD digest
+                     STORM-PRINCIPAL-TO-LOCAL-PLUGIN "backtype.storm.security.auth.DefaultPrincipalToLocal"
                      NIMBUS-THRIFT-PORT 6666}
           expected-acls nimbus/NIMBUS-ZK-ACLS
           fake-inimbus (reify INimbus (getForcedScheduler [this] nil))]
@@ -1272,10 +1248,11 @@
                  mk-authorization-handler nil
                  cluster/mk-storm-cluster-state nil
                  nimbus/file-cache-map nil
+                 nimbus/mk-blob-cache-map nil
+                 nimbus/mk-bloblist-cache-map nil
                  uptime-computer nil
                  new-instance nil
                  mk-timer nil
-                 nimbus/mk-code-distributor nil
                  zk-leader-elector nil
                  nimbus/mk-scheduler nil]
         (nimbus/nimbus-data auth-conf fake-inimbus)

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/test/clj/backtype/storm/security/auth/ReqContext_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/security/auth/ReqContext_test.clj b/storm-core/test/clj/backtype/storm/security/auth/ReqContext_test.clj
index ea45ddc..18d4ada 100644
--- a/storm-core/test/clj/backtype/storm/security/auth/ReqContext_test.clj
+++ b/storm-core/test/clj/backtype/storm/security/auth/ReqContext_test.clj
@@ -68,5 +68,6 @@
     (.setSubject rc s)
     (is (not (nil? (.principal rc))))
     (is (= (-> rc .principal .getName) principal-name))
+    (.setSubject rc nil)
   )
 )

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/test/clj/backtype/storm/supervisor_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/supervisor_test.clj b/storm-core/test/clj/backtype/storm/supervisor_test.clj
index 04c8600..776ad6e 100644
--- a/storm-core/test/clj/backtype/storm/supervisor_test.clj
+++ b/storm-core/test/clj/backtype/storm/supervisor_test.clj
@@ -324,7 +324,8 @@
                      set-worker-user! nil
                      supervisor/jlp nil
                      worker-artifacts-root "/tmp/workers-artifacts"
-                     supervisor/write-log-metadata! nil]
+                     supervisor/write-log-metadata! nil
+                     supervisor/create-blobstore-links nil]
             (supervisor/launch-worker mock-supervisor
                                       mock-storm-id
                                       mock-port
@@ -346,8 +347,9 @@
                      launch-process nil
                      set-worker-user! nil
                      supervisor/jlp nil
-                     worker-artifacts-root "/tmp/workers-artifacts"
-                     supervisor/write-log-metadata! nil]
+                     supervisor/write-log-metadata! nil
+                     supervisor/create-blobstore-links nil
+                     worker-artifacts-root "/tmp/workers-artifacts"]
             (supervisor/launch-worker mock-supervisor
                                       mock-storm-id
                                       mock-port
@@ -367,7 +369,8 @@
                      set-worker-user! nil
                      supervisor/write-log-metadata! nil
                      launch-process nil
-                     current-classpath (str file-path-separator "base")]
+                     current-classpath (str file-path-separator "base")
+                     supervisor/create-blobstore-links nil]
                     (supervisor/launch-worker mock-supervisor
                                               mock-storm-id
                                               mock-port
@@ -388,7 +391,8 @@
                      launch-process nil
                      set-worker-user! nil
                      supervisor/write-log-metadata! nil
-                     current-classpath (str file-path-separator "base")]
+                     current-classpath (str file-path-separator "base")
+                     supervisor/create-blobstore-links nil]
                     (supervisor/launch-worker mock-supervisor
                                               mock-storm-id
                                               mock-port
@@ -540,8 +544,8 @@
                  cluster/mk-storm-cluster-state nil
                  supervisor-state nil
                  local-hostname nil
-                 supervisor/mk-code-distributor nil
-                 mk-timer nil]
+                 mk-timer nil
+                 supervisor-local-dir nil]
         (supervisor/supervisor-data auth-conf nil fake-isupervisor)
         (verify-call-times-for cluster/mk-storm-cluster-state 1)
         (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/test/jvm/backtype/storm/blobstore/BlobStoreTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/blobstore/BlobStoreTest.java b/storm-core/test/jvm/backtype/storm/blobstore/BlobStoreTest.java
new file mode 100644
index 0000000..388b491
--- /dev/null
+++ b/storm-core/test/jvm/backtype/storm/blobstore/BlobStoreTest.java
@@ -0,0 +1,461 @@
+/**
+ * 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.SettableBlobMeta;
+
+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.junit.After;
+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.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);
+  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);
+  }
+
+  // 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 {
+    InputStream in = store.getBlob(key, who);
+    try {
+      return in.read();
+    } finally {
+      in.close();
+    }
+  }
+
+  public static int readInt(BlobStore store, String key)
+          throws IOException, KeyNotFoundException, AuthorizationException {
+    return readInt(store, null, key);
+  }
+
+  public static void readAssertEquals(BlobStore store, String key, int value)
+          throws IOException, KeyNotFoundException, AuthorizationException {
+    assertEquals(value, readInt(store, key));
+  }
+
+  // Checks for assertion when we turn on security
+  public void readAssertEqualsWithAuth(BlobStore store, Subject who, String key, int value)
+          throws IOException, KeyNotFoundException, AuthorizationException {
+   assertEquals(value, readInt(store, who, key));
+  }
+
+  private LocalFsBlobStore initLocalFs() {
+    LocalFsBlobStore store = new LocalFsBlobStore();
+    // Spy object that tries to mock the real object store
+    LocalFsBlobStore spy = spy(store);
+    Mockito.doNothing().when(spy).checkForBlobUpdate("test");
+    Mockito.doNothing().when(spy).checkForBlobUpdate("other");
+    Mockito.doNothing().when(spy).checkForBlobUpdate("test-empty-subject-WE");
+    Mockito.doNothing().when(spy).checkForBlobUpdate("test-empty-subject-DEF");
+    Mockito.doNothing().when(spy).checkForBlobUpdate("test-empty-acls");
+    Map conf = Utils.readStormConfig();
+    conf.put(Config.STORM_LOCAL_DIR, baseFile.getAbsolutePath());
+    conf.put(Config.STORM_PRINCIPAL_TO_LOCAL_PLUGIN,"backtype.storm.security.auth.DefaultPrincipalToLocal");
+    ArrayList<String> zookeeper_list = new ArrayList<>();
+    spy.prepare(conf, null, null);
+    return spy;
+  }
+
+  @Test
+  public void testLocalFsWithAuth() throws Exception {
+    testWithAuthentication(initLocalFs());
+  }
+
+  @Test
+  public void testBasicLocalFs() throws Exception {
+    testBasic(initLocalFs());
+  }
+
+  @Test
+  public void testMultipleLocalFs() throws Exception {
+    testMultiple(initLocalFs());
+  }
+
+  public Subject getSubject(String name) {
+    Subject subject = new Subject();
+    SingleUserPrincipal user = new SingleUserPrincipal(name);
+    subject.getPrincipals().add(user);
+    return subject;
+  }
+
+  // Check for Blobstore with authentication
+  public void testWithAuthentication(BlobStore store) throws Exception {
+    //Test for Nimbus Admin
+    Subject admin = getSubject("admin");
+    assertStoreHasExactly(store);
+    SettableBlobMeta metadata = new SettableBlobMeta(BlobStoreAclHandler.DEFAULT);
+    AtomicOutputStream out = store.createBlob("test", metadata, admin);
+    assertStoreHasExactly(store, "test");
+    out.write(1);
+    out.close();
+    store.deleteBlob("test", admin);
+
+    //Test for Supervisor Admin
+    Subject supervisor = getSubject("supervisor");
+    assertStoreHasExactly(store);
+    metadata = new SettableBlobMeta(BlobStoreAclHandler.DEFAULT);
+    out = store.createBlob("test", metadata, supervisor);
+    assertStoreHasExactly(store, "test");
+    out.write(1);
+    out.close();
+    store.deleteBlob("test", supervisor);
+
+    //Test for Nimbus itself as a user
+    Subject nimbus = getNimbusSubject();
+    assertStoreHasExactly(store);
+    metadata = new SettableBlobMeta(BlobStoreAclHandler.DEFAULT);
+    out = store.createBlob("test", metadata, nimbus);
+    assertStoreHasExactly(store, "test");
+    out.write(1);
+    out.close();
+    store.deleteBlob("test", nimbus);
+
+    // Test with a dummy test_subject for cases where subject !=null (security turned on)
+    Subject who = getSubject("test_subject");
+    assertStoreHasExactly(store);
+
+    // Tests for case when subject != null (security turned on) and
+    // acls for the blob are set to WORLD_EVERYTHING
+    metadata = new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING);
+    out = store.createBlob("test", metadata, who);
+    out.write(1);
+    out.close();
+    assertStoreHasExactly(store, "test");
+    // Testing whether acls are set to WORLD_EVERYTHING
+    assertTrue("ACL does not contain WORLD_EVERYTHING", metadata.toString().contains("AccessControl(type:OTHER, access:7)"));
+    readAssertEqualsWithAuth(store, who, "test", 1);
+
+    LOG.info("Deleting test");
+    store.deleteBlob("test", who);
+    assertStoreHasExactly(store);
+
+    // Tests for case when subject != null (security turned on) and
+    // acls are not set for the blob (DEFAULT)
+    LOG.info("Creating test again");
+    metadata = new SettableBlobMeta(BlobStoreAclHandler.DEFAULT);
+    out = store.createBlob("test", metadata, who);
+    out.write(2);
+    out.close();
+    assertStoreHasExactly(store, "test");
+    // Testing whether acls are set to WORLD_EVERYTHING. Here the acl should not contain WORLD_EVERYTHING because
+    // the subject is neither null nor empty. The ACL should however contain USER_EVERYTHING as user needs to have
+    // complete access to the blob
+    assertTrue("ACL does not contain WORLD_EVERYTHING", !metadata.toString().contains("AccessControl(type:OTHER, access:7)"));
+    readAssertEqualsWithAuth(store, who, "test", 2);
+
+    LOG.info("Updating test");
+    out = store.updateBlob("test", who);
+    out.write(3);
+    out.close();
+    assertStoreHasExactly(store, "test");
+    readAssertEqualsWithAuth(store, who, "test", 3);
+
+    LOG.info("Updating test again");
+    out = store.updateBlob("test", who);
+    out.write(4);
+    out.flush();
+    LOG.info("SLEEPING");
+    Thread.sleep(2);
+    assertStoreHasExactly(store, "test");
+    readAssertEqualsWithAuth(store, who, "test", 3);
+
+    // Test for subject with no principals and acls set to WORLD_EVERYTHING
+    who = new Subject();
+    metadata = new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING);
+    LOG.info("Creating test");
+    out = store.createBlob("test-empty-subject-WE", metadata, who);
+    out.write(2);
+    out.close();
+    assertStoreHasExactly(store, "test-empty-subject-WE", "test");
+    // Testing whether acls are set to WORLD_EVERYTHING
+    assertTrue("ACL does not contain WORLD_EVERYTHING", metadata.toString().contains("AccessControl(type:OTHER, access:7)"));
+    readAssertEqualsWithAuth(store, who, "test-empty-subject-WE", 2);
+
+    // Test for subject with no principals and acls set to DEFAULT
+    who = new Subject();
+    metadata = new SettableBlobMeta(BlobStoreAclHandler.DEFAULT);
+    LOG.info("Creating other");
+    out = store.createBlob("test-empty-subject-DEF", metadata, who);
+    out.write(2);
+    out.close();
+    assertStoreHasExactly(store, "test-empty-subject-DEF", "test", "test-empty-subject-WE");
+    // Testing whether acls are set to WORLD_EVERYTHING
+    assertTrue("ACL does not contain WORLD_EVERYTHING", metadata.toString().contains("AccessControl(type:OTHER, access:7)"));
+    readAssertEqualsWithAuth(store, who, "test-empty-subject-DEF", 2);
+
+    if (store instanceof LocalFsBlobStore) {
+      ((LocalFsBlobStore) store).fullCleanup(1);
+    } else {
+      fail("Error the blobstore is of unknowntype");
+    }
+    try {
+      out.close();
+    } catch (IOException e) {
+      // This is likely to happen when we try to commit something that
+      // was cleaned up.  This is expected and acceptable.
+    }
+  }
+
+  public void testBasic(BlobStore store) throws Exception {
+    assertStoreHasExactly(store);
+    LOG.info("Creating test");
+    // Tests for case when subject == null (security turned off) and
+    // acls for the blob are set to WORLD_EVERYTHING
+    SettableBlobMeta metadata = new SettableBlobMeta(BlobStoreAclHandler
+            .WORLD_EVERYTHING);
+    AtomicOutputStream out = store.createBlob("test", metadata, null);
+    out.write(1);
+    out.close();
+    assertStoreHasExactly(store, "test");
+    // Testing whether acls are set to WORLD_EVERYTHING
+    assertTrue("ACL does not contain WORLD_EVERYTHING", metadata.toString().contains("AccessControl(type:OTHER, access:7)"));
+    readAssertEquals(store, "test", 1);
+
+    LOG.info("Deleting test");
+    store.deleteBlob("test", null);
+    assertStoreHasExactly(store);
+
+    // The following tests are run for both hdfs and local store to test the
+    // update blob interface
+    metadata = new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING);
+    LOG.info("Creating test again");
+    out = store.createBlob("test", metadata, null);
+    out.write(2);
+    out.close();
+    assertStoreHasExactly(store, "test");
+    if (store instanceof LocalFsBlobStore) {
+      assertTrue("ACL does not contain WORLD_EVERYTHING", metadata.toString().contains("AccessControl(type:OTHER, access:7)"));
+    }
+    readAssertEquals(store, "test", 2);
+    LOG.info("Updating test");
+    out = store.updateBlob("test", null);
+    out.write(3);
+    out.close();
+    assertStoreHasExactly(store, "test");
+    readAssertEquals(store, "test", 3);
+
+    LOG.info("Updating test again");
+    out = store.updateBlob("test", null);
+    out.write(4);
+    out.flush();
+    LOG.info("SLEEPING");
+    Thread.sleep(2);
+
+    // Tests for case when subject == null (security turned off) and
+    // acls for the blob are set to DEFAULT (Empty ACL List) only for LocalFsBlobstore
+    if (store instanceof LocalFsBlobStore) {
+      metadata = new SettableBlobMeta(BlobStoreAclHandler.DEFAULT);
+      LOG.info("Creating test for empty acls when security is off");
+      out = store.createBlob("test-empty-acls", metadata, null);
+      LOG.info("metadata {}", metadata);
+      out.write(2);
+      out.close();
+      assertStoreHasExactly(store, "test-empty-acls", "test");
+      // Testing whether acls are set to WORLD_EVERYTHING, Here we are testing only for LocalFsBlobstore
+      // as the HdfsBlobstore gets the subject information of the local system user and behaves as it is
+      // always authenticated.
+      assertTrue("ACL does not contain WORLD_EVERYTHING", metadata.get_acl().toString().contains("OTHER"));
+
+      LOG.info("Deleting test-empty-acls");
+      store.deleteBlob("test-empty-acls", null);
+    }
+
+    if (store instanceof LocalFsBlobStore) {
+      ((LocalFsBlobStore) store).fullCleanup(1);
+    } else {
+      fail("Error the blobstore is of unknowntype");
+    }
+    try {
+      out.close();
+    } catch (IOException e) {
+      // This is likely to happen when we try to commit something that
+      // was cleaned up.  This is expected and acceptable.
+    }
+  }
+
+
+  public void testMultiple(BlobStore store) throws Exception {
+
+    assertStoreHasExactly(store);
+    LOG.info("Creating test");
+    AtomicOutputStream out = store.createBlob("test", new SettableBlobMeta(BlobStoreAclHandler
+        .WORLD_EVERYTHING), null);
+    out.write(1);
+    out.close();
+    assertStoreHasExactly(store, "test");
+    readAssertEquals(store, "test", 1);
+
+    LOG.info("Creating other");
+    out = store.createBlob("other", new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING),
+        null);
+    out.write(2);
+    out.close();
+    assertStoreHasExactly(store, "test", "other");
+    readAssertEquals(store, "test", 1);
+    readAssertEquals(store, "other", 2);
+
+    LOG.info("Updating other");
+    out = store.updateBlob("other", null);
+    out.write(5);
+    out.close();
+    assertStoreHasExactly(store, "test", "other");
+    readAssertEquals(store, "test", 1);
+    readAssertEquals(store, "other", 5);
+
+    LOG.info("Deleting test");
+    store.deleteBlob("test", null);
+    assertStoreHasExactly(store, "other");
+    readAssertEquals(store, "other", 5);
+
+    LOG.info("Creating test again");
+    out = store.createBlob("test", new SettableBlobMeta(BlobStoreAclHandler.WORLD_EVERYTHING),
+        null);
+    out.write(2);
+    out.close();
+    assertStoreHasExactly(store, "test", "other");
+    readAssertEquals(store, "test", 2);
+    readAssertEquals(store, "other", 5);
+
+    LOG.info("Updating test");
+    out = store.updateBlob("test", null);
+    out.write(3);
+    out.close();
+    assertStoreHasExactly(store, "test", "other");
+    readAssertEquals(store, "test", 3);
+    readAssertEquals(store, "other", 5);
+
+    LOG.info("Deleting other");
+    store.deleteBlob("other", null);
+    assertStoreHasExactly(store, "test");
+    readAssertEquals(store, "test", 3);
+
+    LOG.info("Updating test again");
+    out = store.updateBlob("test", null);
+    out.write(4);
+    out.flush();
+    LOG.info("SLEEPING");
+    Thread.sleep(2);
+
+    if (store instanceof LocalFsBlobStore) {
+      ((LocalFsBlobStore) store).fullCleanup(1);
+    } else {
+      fail("Error the blobstore is of unknowntype");
+    }    assertStoreHasExactly(store, "test");
+    readAssertEquals(store, "test", 3);
+    try {
+      out.close();
+    } catch (IOException e) {
+      // This is likely to happen when we try to commit something that
+      // was cleaned up.  This is expected and acceptable.
+    }
+  }
+
+  @Test
+  public void testGetFileLength()
+          throws AuthorizationException, KeyNotFoundException, KeyAlreadyExistsException, IOException {
+    LocalFsBlobStore store = initLocalFs();
+    AtomicOutputStream out = store.createBlob("test", new SettableBlobMeta(BlobStoreAclHandler
+        .WORLD_EVERYTHING), null);
+    out.write(1);
+    out.close();
+    assertEquals(1, store.getBlob("test", null).getFileLength());
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/test/jvm/backtype/storm/blobstore/BlobSynchronizerTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/backtype/storm/blobstore/BlobSynchronizerTest.java b/storm-core/test/jvm/backtype/storm/blobstore/BlobSynchronizerTest.java
new file mode 100644
index 0000000..63f633d
--- /dev/null
+++ b/storm-core/test/jvm/backtype/storm/blobstore/BlobSynchronizerTest.java
@@ -0,0 +1,137 @@
+/**
+ * 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.nimbus.NimbusInfo;
+import backtype.storm.utils.Utils;
+import org.apache.commons.io.FileUtils;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.retry.ExponentialBackoffRetry;
+import org.apache.curator.test.TestingServer;
+import org.apache.zookeeper.server.NIOServerCnxnFactory;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.*;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ *  Unit tests for most of the testable utility methods
+ *  and BlobSynchronizer class methods
+ */
+public class BlobSynchronizerTest {
+  private URI base;
+  private File baseFile;
+  private static Map conf = new HashMap();
+  private NIOServerCnxnFactory factory;
+
+  @Before
+  public void init() throws Exception {
+    initializeConfigs();
+    baseFile = new File("/tmp/blob-store-test-"+ UUID.randomUUID());
+    base = baseFile.toURI();
+  }
+
+  @After
+  public void cleanUp() throws IOException {
+    FileUtils.deleteDirectory(baseFile);
+    if (factory != null) {
+      factory.shutdown();
+    }
+  }
+
+  // Method which initializes nimbus admin
+  public static void initializeConfigs() {
+    conf.put(Config.NIMBUS_ADMINS,"admin");
+    conf.put(Config.NIMBUS_SUPERVISOR_USERS,"supervisor");
+  }
+
+  private LocalFsBlobStore initLocalFs() {
+    LocalFsBlobStore store = new LocalFsBlobStore();
+    Map conf = Utils.readStormConfig();
+    conf.put(Config.STORM_LOCAL_DIR, baseFile.getAbsolutePath());
+    conf.put(Config.STORM_PRINCIPAL_TO_LOCAL_PLUGIN,"backtype.storm.security.auth.DefaultPrincipalToLocal");
+    this.conf = conf;
+    store.prepare(conf, null, null);
+    return store;
+  }
+
+  @Test
+  public void testBlobSynchronizerForKeysToDownload() {
+    BlobStore store = initLocalFs();
+    BlobSynchronizer sync = new BlobSynchronizer(store, conf);
+    // test for keylist to download
+    Set<String> zkSet = new HashSet<String>();
+    zkSet.add("key1");
+    Set<String> blobStoreSet = new HashSet<String>();
+    blobStoreSet.add("key1");
+    Set<String> resultSet = sync.getKeySetToDownload(blobStoreSet, zkSet);
+    assertTrue("Not Empty", resultSet.isEmpty());
+    zkSet.add("key1");
+    blobStoreSet.add("key2");
+    resultSet =  sync.getKeySetToDownload(blobStoreSet, zkSet);
+    assertTrue("Not Empty", resultSet.isEmpty());
+    blobStoreSet.remove("key1");
+    blobStoreSet.remove("key2");
+    zkSet.add("key1");
+    resultSet =  sync.getKeySetToDownload(blobStoreSet, zkSet);
+    assertTrue("Unexpected keys to download", (resultSet.size() == 1) && (resultSet.contains("key1")));
+  }
+
+  @Test
+  public void testGetLatestSequenceNumber() throws Exception {
+    List<String> stateInfoList = new ArrayList<String>();
+    stateInfoList.add("nimbus1:8000-2");
+    stateInfoList.add("nimbus-1:8000-4");
+    assertTrue("Failed to get the latest version", BlobStoreUtils.getLatestSequenceNumber(stateInfoList)==4);
+  }
+
+  @Test
+  public void testNimbodesWithLatestVersionOfBlob() throws Exception {
+    TestingServer server = new TestingServer();
+    CuratorFramework zkClient = CuratorFrameworkFactory.newClient(server.getConnectString(), new ExponentialBackoffRetry(1000, 3));
+    zkClient.start();
+    // Creating nimbus hosts containing latest version of blob
+    zkClient.create().creatingParentContainersIfNeeded().forPath("/blobstore/key1/nimbus1:7800-1");
+    zkClient.create().creatingParentContainersIfNeeded().forPath("/blobstore/key1/nimbus2:7800-2");
+    Set<NimbusInfo> set = BlobStoreUtils.getNimbodesWithLatestSequenceNumberOfBlob(zkClient, "key1");
+    assertEquals("Failed to get the correct nimbus hosts with latest blob version", (set.iterator().next()).getHost(),"nimbus2");
+    zkClient.delete().deletingChildrenIfNeeded().forPath("/blobstore/key1/nimbus1:7800-1");
+    zkClient.delete().deletingChildrenIfNeeded().forPath("/blobstore/key1/nimbus2:7800-2");
+    zkClient.close();
+    server.close();
+  }
+
+  @Test
+  public void testNormalizeVersionInfo () throws Exception {
+    BlobKeySequenceInfo info1 = BlobStoreUtils.normalizeNimbusHostPortSequenceNumberInfo("nimbus1:7800-1");
+    assertTrue(info1.getNimbusHostPort().equals("nimbus1:7800"));
+    assertTrue(info1.getSequenceNumber().equals("1"));
+    BlobKeySequenceInfo info2 = BlobStoreUtils.normalizeNimbusHostPortSequenceNumberInfo("nimbus-1:7800-1");
+    assertTrue(info2.getNimbusHostPort().equals("nimbus-1:7800"));
+    assertTrue(info2.getSequenceNumber().equals("1"));
+  }
+}


[17/17] storm git commit: Added STORM-876 to Changelog

Posted by bo...@apache.org.
Added STORM-876 to Changelog


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

Branch: refs/heads/master
Commit: 6babbb08c43a32a7624e81dfc3a3bfe52db02a33
Parents: bdbec85
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Fri Dec 4 09:03:09 2015 -0600
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Fri Dec 4 09:03:09 2015 -0600

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


http://git-wip-us.apache.org/repos/asf/storm/blob/6babbb08/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 8420f1a..2185747 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 ## 0.11.0
+ * STORM-876: Blobstore/DistCache Support
  * STORM-1361: Apache License missing from two Cassandra files
  * STORM-1218: Use markdown for JavaDoc.
  * STORM-1075: Storm Cassandra connector.


[03/17] storm git commit: Blobstore API STORM- 876

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/py/storm/Nimbus.py
----------------------------------------------------------------------
diff --git a/storm-core/src/py/storm/Nimbus.py b/storm-core/src/py/storm/Nimbus.py
index c2bb9ac..3b680ec 100644
--- a/storm-core/src/py/storm/Nimbus.py
+++ b/storm-core/src/py/storm/Nimbus.py
@@ -151,6 +151,108 @@ class Iface:
     """
     pass
 
+  def beginCreateBlob(self, key, meta):
+    """
+    Parameters:
+     - key
+     - meta
+    """
+    pass
+
+  def beginUpdateBlob(self, key):
+    """
+    Parameters:
+     - key
+    """
+    pass
+
+  def uploadBlobChunk(self, session, chunk):
+    """
+    Parameters:
+     - session
+     - chunk
+    """
+    pass
+
+  def finishBlobUpload(self, session):
+    """
+    Parameters:
+     - session
+    """
+    pass
+
+  def cancelBlobUpload(self, session):
+    """
+    Parameters:
+     - session
+    """
+    pass
+
+  def getBlobMeta(self, key):
+    """
+    Parameters:
+     - key
+    """
+    pass
+
+  def setBlobMeta(self, key, meta):
+    """
+    Parameters:
+     - key
+     - meta
+    """
+    pass
+
+  def beginBlobDownload(self, key):
+    """
+    Parameters:
+     - key
+    """
+    pass
+
+  def downloadBlobChunk(self, session):
+    """
+    Parameters:
+     - session
+    """
+    pass
+
+  def deleteBlob(self, key):
+    """
+    Parameters:
+     - key
+    """
+    pass
+
+  def listBlobs(self, session):
+    """
+    Parameters:
+     - session
+    """
+    pass
+
+  def getBlobReplication(self, key):
+    """
+    Parameters:
+     - key
+    """
+    pass
+
+  def updateBlobReplication(self, key, replication):
+    """
+    Parameters:
+     - key
+     - replication
+    """
+    pass
+
+  def createStateInZookeeper(self, key):
+    """
+    Parameters:
+     - key
+    """
+    pass
+
   def beginFileUpload(self):
     pass
 
@@ -728,18 +830,25 @@ class Client(Iface):
       raise result.aze
     return
 
-  def beginFileUpload(self):
-    self.send_beginFileUpload()
-    return self.recv_beginFileUpload()
-
-  def send_beginFileUpload(self):
-    self._oprot.writeMessageBegin('beginFileUpload', TMessageType.CALL, self._seqid)
-    args = beginFileUpload_args()
+  def beginCreateBlob(self, key, meta):
+    """
+    Parameters:
+     - key
+     - meta
+    """
+    self.send_beginCreateBlob(key, meta)
+    return self.recv_beginCreateBlob()
+
+  def send_beginCreateBlob(self, key, meta):
+    self._oprot.writeMessageBegin('beginCreateBlob', TMessageType.CALL, self._seqid)
+    args = beginCreateBlob_args()
+    args.key = key
+    args.meta = meta
     args.write(self._oprot)
     self._oprot.writeMessageEnd()
     self._oprot.trans.flush()
 
-  def recv_beginFileUpload(self):
+  def recv_beginCreateBlob(self):
     iprot = self._iprot
     (fname, mtype, rseqid) = iprot.readMessageBegin()
     if mtype == TMessageType.EXCEPTION:
@@ -747,34 +856,34 @@ class Client(Iface):
       x.read(iprot)
       iprot.readMessageEnd()
       raise x
-    result = beginFileUpload_result()
+    result = beginCreateBlob_result()
     result.read(iprot)
     iprot.readMessageEnd()
     if result.success is not None:
       return result.success
     if result.aze is not None:
       raise result.aze
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "beginFileUpload failed: unknown result")
+    if result.kae is not None:
+      raise result.kae
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "beginCreateBlob failed: unknown result")
 
-  def uploadChunk(self, location, chunk):
+  def beginUpdateBlob(self, key):
     """
     Parameters:
-     - location
-     - chunk
+     - key
     """
-    self.send_uploadChunk(location, chunk)
-    self.recv_uploadChunk()
+    self.send_beginUpdateBlob(key)
+    return self.recv_beginUpdateBlob()
 
-  def send_uploadChunk(self, location, chunk):
-    self._oprot.writeMessageBegin('uploadChunk', TMessageType.CALL, self._seqid)
-    args = uploadChunk_args()
-    args.location = location
-    args.chunk = chunk
+  def send_beginUpdateBlob(self, key):
+    self._oprot.writeMessageBegin('beginUpdateBlob', TMessageType.CALL, self._seqid)
+    args = beginUpdateBlob_args()
+    args.key = key
     args.write(self._oprot)
     self._oprot.writeMessageEnd()
     self._oprot.trans.flush()
 
-  def recv_uploadChunk(self):
+  def recv_beginUpdateBlob(self):
     iprot = self._iprot
     (fname, mtype, rseqid) = iprot.readMessageBegin()
     if mtype == TMessageType.EXCEPTION:
@@ -782,30 +891,36 @@ class Client(Iface):
       x.read(iprot)
       iprot.readMessageEnd()
       raise x
-    result = uploadChunk_result()
+    result = beginUpdateBlob_result()
     result.read(iprot)
     iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
     if result.aze is not None:
       raise result.aze
-    return
+    if result.knf is not None:
+      raise result.knf
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "beginUpdateBlob failed: unknown result")
 
-  def finishFileUpload(self, location):
+  def uploadBlobChunk(self, session, chunk):
     """
     Parameters:
-     - location
+     - session
+     - chunk
     """
-    self.send_finishFileUpload(location)
-    self.recv_finishFileUpload()
+    self.send_uploadBlobChunk(session, chunk)
+    self.recv_uploadBlobChunk()
 
-  def send_finishFileUpload(self, location):
-    self._oprot.writeMessageBegin('finishFileUpload', TMessageType.CALL, self._seqid)
-    args = finishFileUpload_args()
-    args.location = location
+  def send_uploadBlobChunk(self, session, chunk):
+    self._oprot.writeMessageBegin('uploadBlobChunk', TMessageType.CALL, self._seqid)
+    args = uploadBlobChunk_args()
+    args.session = session
+    args.chunk = chunk
     args.write(self._oprot)
     self._oprot.writeMessageEnd()
     self._oprot.trans.flush()
 
-  def recv_finishFileUpload(self):
+  def recv_uploadBlobChunk(self):
     iprot = self._iprot
     (fname, mtype, rseqid) = iprot.readMessageBegin()
     if mtype == TMessageType.EXCEPTION:
@@ -813,30 +928,30 @@ class Client(Iface):
       x.read(iprot)
       iprot.readMessageEnd()
       raise x
-    result = finishFileUpload_result()
+    result = uploadBlobChunk_result()
     result.read(iprot)
     iprot.readMessageEnd()
     if result.aze is not None:
       raise result.aze
     return
 
-  def beginFileDownload(self, file):
+  def finishBlobUpload(self, session):
     """
     Parameters:
-     - file
+     - session
     """
-    self.send_beginFileDownload(file)
-    return self.recv_beginFileDownload()
+    self.send_finishBlobUpload(session)
+    self.recv_finishBlobUpload()
 
-  def send_beginFileDownload(self, file):
-    self._oprot.writeMessageBegin('beginFileDownload', TMessageType.CALL, self._seqid)
-    args = beginFileDownload_args()
-    args.file = file
+  def send_finishBlobUpload(self, session):
+    self._oprot.writeMessageBegin('finishBlobUpload', TMessageType.CALL, self._seqid)
+    args = finishBlobUpload_args()
+    args.session = session
     args.write(self._oprot)
     self._oprot.writeMessageEnd()
     self._oprot.trans.flush()
 
-  def recv_beginFileDownload(self):
+  def recv_finishBlobUpload(self):
     iprot = self._iprot
     (fname, mtype, rseqid) = iprot.readMessageBegin()
     if mtype == TMessageType.EXCEPTION:
@@ -844,32 +959,30 @@ class Client(Iface):
       x.read(iprot)
       iprot.readMessageEnd()
       raise x
-    result = beginFileDownload_result()
+    result = finishBlobUpload_result()
     result.read(iprot)
     iprot.readMessageEnd()
-    if result.success is not None:
-      return result.success
     if result.aze is not None:
       raise result.aze
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "beginFileDownload failed: unknown result")
+    return
 
-  def downloadChunk(self, id):
+  def cancelBlobUpload(self, session):
     """
     Parameters:
-     - id
+     - session
     """
-    self.send_downloadChunk(id)
-    return self.recv_downloadChunk()
+    self.send_cancelBlobUpload(session)
+    self.recv_cancelBlobUpload()
 
-  def send_downloadChunk(self, id):
-    self._oprot.writeMessageBegin('downloadChunk', TMessageType.CALL, self._seqid)
-    args = downloadChunk_args()
-    args.id = id
+  def send_cancelBlobUpload(self, session):
+    self._oprot.writeMessageBegin('cancelBlobUpload', TMessageType.CALL, self._seqid)
+    args = cancelBlobUpload_args()
+    args.session = session
     args.write(self._oprot)
     self._oprot.writeMessageEnd()
     self._oprot.trans.flush()
 
-  def recv_downloadChunk(self):
+  def recv_cancelBlobUpload(self):
     iprot = self._iprot
     (fname, mtype, rseqid) = iprot.readMessageBegin()
     if mtype == TMessageType.EXCEPTION:
@@ -877,27 +990,30 @@ class Client(Iface):
       x.read(iprot)
       iprot.readMessageEnd()
       raise x
-    result = downloadChunk_result()
+    result = cancelBlobUpload_result()
     result.read(iprot)
     iprot.readMessageEnd()
-    if result.success is not None:
-      return result.success
     if result.aze is not None:
       raise result.aze
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "downloadChunk failed: unknown result")
+    return
 
-  def getNimbusConf(self):
-    self.send_getNimbusConf()
-    return self.recv_getNimbusConf()
+  def getBlobMeta(self, key):
+    """
+    Parameters:
+     - key
+    """
+    self.send_getBlobMeta(key)
+    return self.recv_getBlobMeta()
 
-  def send_getNimbusConf(self):
-    self._oprot.writeMessageBegin('getNimbusConf', TMessageType.CALL, self._seqid)
-    args = getNimbusConf_args()
+  def send_getBlobMeta(self, key):
+    self._oprot.writeMessageBegin('getBlobMeta', TMessageType.CALL, self._seqid)
+    args = getBlobMeta_args()
+    args.key = key
     args.write(self._oprot)
     self._oprot.writeMessageEnd()
     self._oprot.trans.flush()
 
-  def recv_getNimbusConf(self):
+  def recv_getBlobMeta(self):
     iprot = self._iprot
     (fname, mtype, rseqid) = iprot.readMessageBegin()
     if mtype == TMessageType.EXCEPTION:
@@ -905,27 +1021,36 @@ class Client(Iface):
       x.read(iprot)
       iprot.readMessageEnd()
       raise x
-    result = getNimbusConf_result()
+    result = getBlobMeta_result()
     result.read(iprot)
     iprot.readMessageEnd()
     if result.success is not None:
       return result.success
     if result.aze is not None:
       raise result.aze
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "getNimbusConf failed: unknown result")
-
-  def getClusterInfo(self):
-    self.send_getClusterInfo()
-    return self.recv_getClusterInfo()
+    if result.knf is not None:
+      raise result.knf
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getBlobMeta failed: unknown result")
 
-  def send_getClusterInfo(self):
-    self._oprot.writeMessageBegin('getClusterInfo', TMessageType.CALL, self._seqid)
-    args = getClusterInfo_args()
+  def setBlobMeta(self, key, meta):
+    """
+    Parameters:
+     - key
+     - meta
+    """
+    self.send_setBlobMeta(key, meta)
+    self.recv_setBlobMeta()
+
+  def send_setBlobMeta(self, key, meta):
+    self._oprot.writeMessageBegin('setBlobMeta', TMessageType.CALL, self._seqid)
+    args = setBlobMeta_args()
+    args.key = key
+    args.meta = meta
     args.write(self._oprot)
     self._oprot.writeMessageEnd()
     self._oprot.trans.flush()
 
-  def recv_getClusterInfo(self):
+  def recv_setBlobMeta(self):
     iprot = self._iprot
     (fname, mtype, rseqid) = iprot.readMessageBegin()
     if mtype == TMessageType.EXCEPTION:
@@ -933,32 +1058,32 @@ class Client(Iface):
       x.read(iprot)
       iprot.readMessageEnd()
       raise x
-    result = getClusterInfo_result()
+    result = setBlobMeta_result()
     result.read(iprot)
     iprot.readMessageEnd()
-    if result.success is not None:
-      return result.success
     if result.aze is not None:
       raise result.aze
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "getClusterInfo failed: unknown result")
+    if result.knf is not None:
+      raise result.knf
+    return
 
-  def getTopologyInfo(self, id):
+  def beginBlobDownload(self, key):
     """
     Parameters:
-     - id
+     - key
     """
-    self.send_getTopologyInfo(id)
-    return self.recv_getTopologyInfo()
+    self.send_beginBlobDownload(key)
+    return self.recv_beginBlobDownload()
 
-  def send_getTopologyInfo(self, id):
-    self._oprot.writeMessageBegin('getTopologyInfo', TMessageType.CALL, self._seqid)
-    args = getTopologyInfo_args()
-    args.id = id
+  def send_beginBlobDownload(self, key):
+    self._oprot.writeMessageBegin('beginBlobDownload', TMessageType.CALL, self._seqid)
+    args = beginBlobDownload_args()
+    args.key = key
     args.write(self._oprot)
     self._oprot.writeMessageEnd()
     self._oprot.trans.flush()
 
-  def recv_getTopologyInfo(self):
+  def recv_beginBlobDownload(self):
     iprot = self._iprot
     (fname, mtype, rseqid) = iprot.readMessageBegin()
     if mtype == TMessageType.EXCEPTION:
@@ -966,36 +1091,34 @@ class Client(Iface):
       x.read(iprot)
       iprot.readMessageEnd()
       raise x
-    result = getTopologyInfo_result()
+    result = beginBlobDownload_result()
     result.read(iprot)
     iprot.readMessageEnd()
     if result.success is not None:
       return result.success
-    if result.e is not None:
-      raise result.e
     if result.aze is not None:
       raise result.aze
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyInfo failed: unknown result")
+    if result.knf is not None:
+      raise result.knf
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "beginBlobDownload failed: unknown result")
 
-  def getTopologyInfoWithOpts(self, id, options):
+  def downloadBlobChunk(self, session):
     """
     Parameters:
-     - id
-     - options
+     - session
     """
-    self.send_getTopologyInfoWithOpts(id, options)
-    return self.recv_getTopologyInfoWithOpts()
+    self.send_downloadBlobChunk(session)
+    return self.recv_downloadBlobChunk()
 
-  def send_getTopologyInfoWithOpts(self, id, options):
-    self._oprot.writeMessageBegin('getTopologyInfoWithOpts', TMessageType.CALL, self._seqid)
-    args = getTopologyInfoWithOpts_args()
-    args.id = id
-    args.options = options
+  def send_downloadBlobChunk(self, session):
+    self._oprot.writeMessageBegin('downloadBlobChunk', TMessageType.CALL, self._seqid)
+    args = downloadBlobChunk_args()
+    args.session = session
     args.write(self._oprot)
     self._oprot.writeMessageEnd()
     self._oprot.trans.flush()
 
-  def recv_getTopologyInfoWithOpts(self):
+  def recv_downloadBlobChunk(self):
     iprot = self._iprot
     (fname, mtype, rseqid) = iprot.readMessageBegin()
     if mtype == TMessageType.EXCEPTION:
@@ -1003,38 +1126,32 @@ class Client(Iface):
       x.read(iprot)
       iprot.readMessageEnd()
       raise x
-    result = getTopologyInfoWithOpts_result()
+    result = downloadBlobChunk_result()
     result.read(iprot)
     iprot.readMessageEnd()
     if result.success is not None:
       return result.success
-    if result.e is not None:
-      raise result.e
     if result.aze is not None:
       raise result.aze
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyInfoWithOpts failed: unknown result")
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "downloadBlobChunk failed: unknown result")
 
-  def getTopologyPageInfo(self, id, window, is_include_sys):
+  def deleteBlob(self, key):
     """
     Parameters:
-     - id
-     - window
-     - is_include_sys
+     - key
     """
-    self.send_getTopologyPageInfo(id, window, is_include_sys)
-    return self.recv_getTopologyPageInfo()
+    self.send_deleteBlob(key)
+    self.recv_deleteBlob()
 
-  def send_getTopologyPageInfo(self, id, window, is_include_sys):
-    self._oprot.writeMessageBegin('getTopologyPageInfo', TMessageType.CALL, self._seqid)
-    args = getTopologyPageInfo_args()
-    args.id = id
-    args.window = window
-    args.is_include_sys = is_include_sys
+  def send_deleteBlob(self, key):
+    self._oprot.writeMessageBegin('deleteBlob', TMessageType.CALL, self._seqid)
+    args = deleteBlob_args()
+    args.key = key
     args.write(self._oprot)
     self._oprot.writeMessageEnd()
     self._oprot.trans.flush()
 
-  def recv_getTopologyPageInfo(self):
+  def recv_deleteBlob(self):
     iprot = self._iprot
     (fname, mtype, rseqid) = iprot.readMessageBegin()
     if mtype == TMessageType.EXCEPTION:
@@ -1042,40 +1159,32 @@ class Client(Iface):
       x.read(iprot)
       iprot.readMessageEnd()
       raise x
-    result = getTopologyPageInfo_result()
+    result = deleteBlob_result()
     result.read(iprot)
     iprot.readMessageEnd()
-    if result.success is not None:
-      return result.success
-    if result.e is not None:
-      raise result.e
     if result.aze is not None:
       raise result.aze
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyPageInfo failed: unknown result")
+    if result.knf is not None:
+      raise result.knf
+    return
 
-  def getComponentPageInfo(self, topology_id, component_id, window, is_include_sys):
+  def listBlobs(self, session):
     """
     Parameters:
-     - topology_id
-     - component_id
-     - window
-     - is_include_sys
+     - session
     """
-    self.send_getComponentPageInfo(topology_id, component_id, window, is_include_sys)
-    return self.recv_getComponentPageInfo()
+    self.send_listBlobs(session)
+    return self.recv_listBlobs()
 
-  def send_getComponentPageInfo(self, topology_id, component_id, window, is_include_sys):
-    self._oprot.writeMessageBegin('getComponentPageInfo', TMessageType.CALL, self._seqid)
-    args = getComponentPageInfo_args()
-    args.topology_id = topology_id
-    args.component_id = component_id
-    args.window = window
-    args.is_include_sys = is_include_sys
+  def send_listBlobs(self, session):
+    self._oprot.writeMessageBegin('listBlobs', TMessageType.CALL, self._seqid)
+    args = listBlobs_args()
+    args.session = session
     args.write(self._oprot)
     self._oprot.writeMessageEnd()
     self._oprot.trans.flush()
 
-  def recv_getComponentPageInfo(self):
+  def recv_listBlobs(self):
     iprot = self._iprot
     (fname, mtype, rseqid) = iprot.readMessageBegin()
     if mtype == TMessageType.EXCEPTION:
@@ -1083,34 +1192,30 @@ class Client(Iface):
       x.read(iprot)
       iprot.readMessageEnd()
       raise x
-    result = getComponentPageInfo_result()
+    result = listBlobs_result()
     result.read(iprot)
     iprot.readMessageEnd()
     if result.success is not None:
       return result.success
-    if result.e is not None:
-      raise result.e
-    if result.aze is not None:
-      raise result.aze
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "getComponentPageInfo failed: unknown result")
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "listBlobs failed: unknown result")
 
-  def getTopologyConf(self, id):
+  def getBlobReplication(self, key):
     """
     Parameters:
-     - id
+     - key
     """
-    self.send_getTopologyConf(id)
-    return self.recv_getTopologyConf()
+    self.send_getBlobReplication(key)
+    return self.recv_getBlobReplication()
 
-  def send_getTopologyConf(self, id):
-    self._oprot.writeMessageBegin('getTopologyConf', TMessageType.CALL, self._seqid)
-    args = getTopologyConf_args()
-    args.id = id
+  def send_getBlobReplication(self, key):
+    self._oprot.writeMessageBegin('getBlobReplication', TMessageType.CALL, self._seqid)
+    args = getBlobReplication_args()
+    args.key = key
     args.write(self._oprot)
     self._oprot.writeMessageEnd()
     self._oprot.trans.flush()
 
-  def recv_getTopologyConf(self):
+  def recv_getBlobReplication(self):
     iprot = self._iprot
     (fname, mtype, rseqid) = iprot.readMessageBegin()
     if mtype == TMessageType.EXCEPTION:
@@ -1118,36 +1223,36 @@ class Client(Iface):
       x.read(iprot)
       iprot.readMessageEnd()
       raise x
-    result = getTopologyConf_result()
+    result = getBlobReplication_result()
     result.read(iprot)
     iprot.readMessageEnd()
     if result.success is not None:
       return result.success
-    if result.e is not None:
-      raise result.e
     if result.aze is not None:
       raise result.aze
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyConf failed: unknown result")
+    if result.knf is not None:
+      raise result.knf
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getBlobReplication failed: unknown result")
 
-  def getTopology(self, id):
+  def updateBlobReplication(self, key, replication):
     """
-    Returns the compiled topology that contains ackers and metrics consumsers. Compare {@link #getUserTopology(String id)}.
-
     Parameters:
-     - id
+     - key
+     - replication
     """
-    self.send_getTopology(id)
-    return self.recv_getTopology()
-
-  def send_getTopology(self, id):
-    self._oprot.writeMessageBegin('getTopology', TMessageType.CALL, self._seqid)
-    args = getTopology_args()
-    args.id = id
+    self.send_updateBlobReplication(key, replication)
+    return self.recv_updateBlobReplication()
+
+  def send_updateBlobReplication(self, key, replication):
+    self._oprot.writeMessageBegin('updateBlobReplication', TMessageType.CALL, self._seqid)
+    args = updateBlobReplication_args()
+    args.key = key
+    args.replication = replication
     args.write(self._oprot)
     self._oprot.writeMessageEnd()
     self._oprot.trans.flush()
 
-  def recv_getTopology(self):
+  def recv_updateBlobReplication(self):
     iprot = self._iprot
     (fname, mtype, rseqid) = iprot.readMessageBegin()
     if mtype == TMessageType.EXCEPTION:
@@ -1155,36 +1260,34 @@ class Client(Iface):
       x.read(iprot)
       iprot.readMessageEnd()
       raise x
-    result = getTopology_result()
+    result = updateBlobReplication_result()
     result.read(iprot)
     iprot.readMessageEnd()
     if result.success is not None:
       return result.success
-    if result.e is not None:
-      raise result.e
     if result.aze is not None:
       raise result.aze
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopology failed: unknown result")
+    if result.knf is not None:
+      raise result.knf
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "updateBlobReplication failed: unknown result")
 
-  def getUserTopology(self, id):
+  def createStateInZookeeper(self, key):
     """
-    Returns the user specified topology as submitted originally. Compare {@link #getTopology(String id)}.
-
     Parameters:
-     - id
+     - key
     """
-    self.send_getUserTopology(id)
-    return self.recv_getUserTopology()
+    self.send_createStateInZookeeper(key)
+    self.recv_createStateInZookeeper()
 
-  def send_getUserTopology(self, id):
-    self._oprot.writeMessageBegin('getUserTopology', TMessageType.CALL, self._seqid)
-    args = getUserTopology_args()
-    args.id = id
+  def send_createStateInZookeeper(self, key):
+    self._oprot.writeMessageBegin('createStateInZookeeper', TMessageType.CALL, self._seqid)
+    args = createStateInZookeeper_args()
+    args.key = key
     args.write(self._oprot)
     self._oprot.writeMessageEnd()
     self._oprot.trans.flush()
 
-  def recv_getUserTopology(self):
+  def recv_createStateInZookeeper(self):
     iprot = self._iprot
     (fname, mtype, rseqid) = iprot.readMessageBegin()
     if mtype == TMessageType.EXCEPTION:
@@ -1192,34 +1295,58 @@ class Client(Iface):
       x.read(iprot)
       iprot.readMessageEnd()
       raise x
-    result = getUserTopology_result()
+    result = createStateInZookeeper_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    return
+
+  def beginFileUpload(self):
+    self.send_beginFileUpload()
+    return self.recv_beginFileUpload()
+
+  def send_beginFileUpload(self):
+    self._oprot.writeMessageBegin('beginFileUpload', TMessageType.CALL, self._seqid)
+    args = beginFileUpload_args()
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_beginFileUpload(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = beginFileUpload_result()
     result.read(iprot)
     iprot.readMessageEnd()
     if result.success is not None:
       return result.success
-    if result.e is not None:
-      raise result.e
     if result.aze is not None:
       raise result.aze
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "getUserTopology failed: unknown result")
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "beginFileUpload failed: unknown result")
 
-  def getTopologyHistory(self, user):
+  def uploadChunk(self, location, chunk):
     """
     Parameters:
-     - user
+     - location
+     - chunk
     """
-    self.send_getTopologyHistory(user)
-    return self.recv_getTopologyHistory()
+    self.send_uploadChunk(location, chunk)
+    self.recv_uploadChunk()
 
-  def send_getTopologyHistory(self, user):
-    self._oprot.writeMessageBegin('getTopologyHistory', TMessageType.CALL, self._seqid)
-    args = getTopologyHistory_args()
-    args.user = user
+  def send_uploadChunk(self, location, chunk):
+    self._oprot.writeMessageBegin('uploadChunk', TMessageType.CALL, self._seqid)
+    args = uploadChunk_args()
+    args.location = location
+    args.chunk = chunk
     args.write(self._oprot)
     self._oprot.writeMessageEnd()
     self._oprot.trans.flush()
 
-  def recv_getTopologyHistory(self):
+  def recv_uploadChunk(self):
     iprot = self._iprot
     (fname, mtype, rseqid) = iprot.readMessageBegin()
     if mtype == TMessageType.EXCEPTION:
@@ -1227,753 +1354,3612 @@ class Client(Iface):
       x.read(iprot)
       iprot.readMessageEnd()
       raise x
-    result = getTopologyHistory_result()
+    result = uploadChunk_result()
     result.read(iprot)
     iprot.readMessageEnd()
-    if result.success is not None:
-      return result.success
     if result.aze is not None:
       raise result.aze
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyHistory failed: unknown result")
+    return
 
+  def finishFileUpload(self, location):
+    """
+    Parameters:
+     - location
+    """
+    self.send_finishFileUpload(location)
+    self.recv_finishFileUpload()
 
-class Processor(Iface, TProcessor):
-  def __init__(self, handler):
-    self._handler = handler
-    self._processMap = {}
-    self._processMap["submitTopology"] = Processor.process_submitTopology
-    self._processMap["submitTopologyWithOpts"] = Processor.process_submitTopologyWithOpts
-    self._processMap["killTopology"] = Processor.process_killTopology
-    self._processMap["killTopologyWithOpts"] = Processor.process_killTopologyWithOpts
-    self._processMap["activate"] = Processor.process_activate
-    self._processMap["deactivate"] = Processor.process_deactivate
-    self._processMap["rebalance"] = Processor.process_rebalance
-    self._processMap["setLogConfig"] = Processor.process_setLogConfig
-    self._processMap["getLogConfig"] = Processor.process_getLogConfig
-    self._processMap["debug"] = Processor.process_debug
-    self._processMap["setWorkerProfiler"] = Processor.process_setWorkerProfiler
-    self._processMap["getComponentPendingProfileActions"] = Processor.process_getComponentPendingProfileActions
-    self._processMap["uploadNewCredentials"] = Processor.process_uploadNewCredentials
-    self._processMap["beginFileUpload"] = Processor.process_beginFileUpload
-    self._processMap["uploadChunk"] = Processor.process_uploadChunk
-    self._processMap["finishFileUpload"] = Processor.process_finishFileUpload
-    self._processMap["beginFileDownload"] = Processor.process_beginFileDownload
-    self._processMap["downloadChunk"] = Processor.process_downloadChunk
-    self._processMap["getNimbusConf"] = Processor.process_getNimbusConf
-    self._processMap["getClusterInfo"] = Processor.process_getClusterInfo
-    self._processMap["getTopologyInfo"] = Processor.process_getTopologyInfo
-    self._processMap["getTopologyInfoWithOpts"] = Processor.process_getTopologyInfoWithOpts
-    self._processMap["getTopologyPageInfo"] = Processor.process_getTopologyPageInfo
-    self._processMap["getComponentPageInfo"] = Processor.process_getComponentPageInfo
-    self._processMap["getTopologyConf"] = Processor.process_getTopologyConf
-    self._processMap["getTopology"] = Processor.process_getTopology
-    self._processMap["getUserTopology"] = Processor.process_getUserTopology
-    self._processMap["getTopologyHistory"] = Processor.process_getTopologyHistory
+  def send_finishFileUpload(self, location):
+    self._oprot.writeMessageBegin('finishFileUpload', TMessageType.CALL, self._seqid)
+    args = finishFileUpload_args()
+    args.location = location
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
 
-  def process(self, iprot, oprot):
-    (name, type, seqid) = iprot.readMessageBegin()
-    if name not in self._processMap:
-      iprot.skip(TType.STRUCT)
+  def recv_finishFileUpload(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
       iprot.readMessageEnd()
-      x = TApplicationException(TApplicationException.UNKNOWN_METHOD, 'Unknown function %s' % (name))
-      oprot.writeMessageBegin(name, TMessageType.EXCEPTION, seqid)
-      x.write(oprot)
-      oprot.writeMessageEnd()
-      oprot.trans.flush()
-      return
-    else:
-      self._processMap[name](self, seqid, iprot, oprot)
-    return True
-
-  def process_submitTopology(self, seqid, iprot, oprot):
-    args = submitTopology_args()
-    args.read(iprot)
+      raise x
+    result = finishFileUpload_result()
+    result.read(iprot)
     iprot.readMessageEnd()
-    result = submitTopology_result()
-    try:
-      self._handler.submitTopology(args.name, args.uploadedJarLocation, args.jsonConf, args.topology)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except AlreadyAliveException as e:
-      msg_type = TMessageType.REPLY
-      result.e = e
-    except InvalidTopologyException as ite:
-      msg_type = TMessageType.REPLY
-      result.ite = ite
-    except AuthorizationException as aze:
-      msg_type = TMessageType.REPLY
-      result.aze = aze
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("submitTopology", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
+    if result.aze is not None:
+      raise result.aze
+    return
 
-  def process_submitTopologyWithOpts(self, seqid, iprot, oprot):
-    args = submitTopologyWithOpts_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = submitTopologyWithOpts_result()
-    try:
-      self._handler.submitTopologyWithOpts(args.name, args.uploadedJarLocation, args.jsonConf, args.topology, args.options)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except AlreadyAliveException as e:
-      msg_type = TMessageType.REPLY
-      result.e = e
-    except InvalidTopologyException as ite:
-      msg_type = TMessageType.REPLY
-      result.ite = ite
-    except AuthorizationException as aze:
-      msg_type = TMessageType.REPLY
-      result.aze = aze
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("submitTopologyWithOpts", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
+  def beginFileDownload(self, file):
+    """
+    Parameters:
+     - file
+    """
+    self.send_beginFileDownload(file)
+    return self.recv_beginFileDownload()
 
-  def process_killTopology(self, seqid, iprot, oprot):
-    args = killTopology_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = killTopology_result()
-    try:
-      self._handler.killTopology(args.name)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except NotAliveException as e:
-      msg_type = TMessageType.REPLY
-      result.e = e
-    except AuthorizationException as aze:
-      msg_type = TMessageType.REPLY
-      result.aze = aze
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("killTopology", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
+  def send_beginFileDownload(self, file):
+    self._oprot.writeMessageBegin('beginFileDownload', TMessageType.CALL, self._seqid)
+    args = beginFileDownload_args()
+    args.file = file
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
 
-  def process_killTopologyWithOpts(self, seqid, iprot, oprot):
-    args = killTopologyWithOpts_args()
-    args.read(iprot)
+  def recv_beginFileDownload(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = beginFileDownload_result()
+    result.read(iprot)
     iprot.readMessageEnd()
-    result = killTopologyWithOpts_result()
-    try:
-      self._handler.killTopologyWithOpts(args.name, args.options)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except NotAliveException as e:
-      msg_type = TMessageType.REPLY
-      result.e = e
-    except AuthorizationException as aze:
-      msg_type = TMessageType.REPLY
-      result.aze = aze
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("killTopologyWithOpts", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
+    if result.success is not None:
+      return result.success
+    if result.aze is not None:
+      raise result.aze
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "beginFileDownload failed: unknown result")
 
-  def process_activate(self, seqid, iprot, oprot):
-    args = activate_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = activate_result()
-    try:
-      self._handler.activate(args.name)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except NotAliveException as e:
-      msg_type = TMessageType.REPLY
-      result.e = e
-    except AuthorizationException as aze:
-      msg_type = TMessageType.REPLY
-      result.aze = aze
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("activate", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
+  def downloadChunk(self, id):
+    """
+    Parameters:
+     - id
+    """
+    self.send_downloadChunk(id)
+    return self.recv_downloadChunk()
 
-  def process_deactivate(self, seqid, iprot, oprot):
-    args = deactivate_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = deactivate_result()
-    try:
-      self._handler.deactivate(args.name)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except NotAliveException as e:
-      msg_type = TMessageType.REPLY
-      result.e = e
-    except AuthorizationException as aze:
-      msg_type = TMessageType.REPLY
-      result.aze = aze
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("deactivate", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
+  def send_downloadChunk(self, id):
+    self._oprot.writeMessageBegin('downloadChunk', TMessageType.CALL, self._seqid)
+    args = downloadChunk_args()
+    args.id = id
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
 
-  def process_rebalance(self, seqid, iprot, oprot):
-    args = rebalance_args()
-    args.read(iprot)
+  def recv_downloadChunk(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = downloadChunk_result()
+    result.read(iprot)
     iprot.readMessageEnd()
-    result = rebalance_result()
-    try:
-      self._handler.rebalance(args.name, args.options)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except NotAliveException as e:
-      msg_type = TMessageType.REPLY
-      result.e = e
-    except InvalidTopologyException as ite:
-      msg_type = TMessageType.REPLY
-      result.ite = ite
-    except AuthorizationException as aze:
-      msg_type = TMessageType.REPLY
-      result.aze = aze
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("rebalance", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
+    if result.success is not None:
+      return result.success
+    if result.aze is not None:
+      raise result.aze
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "downloadChunk failed: unknown result")
 
-  def process_setLogConfig(self, seqid, iprot, oprot):
-    args = setLogConfig_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = setLogConfig_result()
-    try:
-      self._handler.setLogConfig(args.name, args.config)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("setLogConfig", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
+  def getNimbusConf(self):
+    self.send_getNimbusConf()
+    return self.recv_getNimbusConf()
 
-  def process_getLogConfig(self, seqid, iprot, oprot):
-    args = getLogConfig_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = getLogConfig_result()
-    try:
-      result.success = self._handler.getLogConfig(args.name)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("getLogConfig", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
+  def send_getNimbusConf(self):
+    self._oprot.writeMessageBegin('getNimbusConf', TMessageType.CALL, self._seqid)
+    args = getNimbusConf_args()
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
 
-  def process_debug(self, seqid, iprot, oprot):
-    args = debug_args()
-    args.read(iprot)
+  def recv_getNimbusConf(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = getNimbusConf_result()
+    result.read(iprot)
     iprot.readMessageEnd()
-    result = debug_result()
-    try:
-      self._handler.debug(args.name, args.component, args.enable, args.samplingPercentage)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except NotAliveException as e:
-      msg_type = TMessageType.REPLY
-      result.e = e
-    except AuthorizationException as aze:
-      msg_type = TMessageType.REPLY
-      result.aze = aze
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("debug", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
+    if result.success is not None:
+      return result.success
+    if result.aze is not None:
+      raise result.aze
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getNimbusConf failed: unknown result")
 
-  def process_setWorkerProfiler(self, seqid, iprot, oprot):
-    args = setWorkerProfiler_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = setWorkerProfiler_result()
-    try:
-      self._handler.setWorkerProfiler(args.id, args.profileRequest)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("setWorkerProfiler", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
+  def getClusterInfo(self):
+    self.send_getClusterInfo()
+    return self.recv_getClusterInfo()
 
-  def process_getComponentPendingProfileActions(self, seqid, iprot, oprot):
-    args = getComponentPendingProfileActions_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = getComponentPendingProfileActions_result()
-    try:
-      result.success = self._handler.getComponentPendingProfileActions(args.id, args.component_id, args.action)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("getComponentPendingProfileActions", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
+  def send_getClusterInfo(self):
+    self._oprot.writeMessageBegin('getClusterInfo', TMessageType.CALL, self._seqid)
+    args = getClusterInfo_args()
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
 
-  def process_uploadNewCredentials(self, seqid, iprot, oprot):
-    args = uploadNewCredentials_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = uploadNewCredentials_result()
-    try:
-      self._handler.uploadNewCredentials(args.name, args.creds)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except NotAliveException as e:
-      msg_type = TMessageType.REPLY
-      result.e = e
-    except InvalidTopologyException as ite:
-      msg_type = TMessageType.REPLY
-      result.ite = ite
-    except AuthorizationException as aze:
-      msg_type = TMessageType.REPLY
-      result.aze = aze
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("uploadNewCredentials", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
-
-  def process_beginFileUpload(self, seqid, iprot, oprot):
-    args = beginFileUpload_args()
-    args.read(iprot)
+  def recv_getClusterInfo(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = getClusterInfo_result()
+    result.read(iprot)
     iprot.readMessageEnd()
-    result = beginFileUpload_result()
-    try:
-      result.success = self._handler.beginFileUpload()
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except AuthorizationException as aze:
-      msg_type = TMessageType.REPLY
-      result.aze = aze
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("beginFileUpload", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
+    if result.success is not None:
+      return result.success
+    if result.aze is not None:
+      raise result.aze
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getClusterInfo failed: unknown result")
 
-  def process_uploadChunk(self, seqid, iprot, oprot):
-    args = uploadChunk_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = uploadChunk_result()
-    try:
-      self._handler.uploadChunk(args.location, args.chunk)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except AuthorizationException as aze:
-      msg_type = TMessageType.REPLY
-      result.aze = aze
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("uploadChunk", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
+  def getTopologyInfo(self, id):
+    """
+    Parameters:
+     - id
+    """
+    self.send_getTopologyInfo(id)
+    return self.recv_getTopologyInfo()
 
-  def process_finishFileUpload(self, seqid, iprot, oprot):
-    args = finishFileUpload_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = finishFileUpload_result()
-    try:
-      self._handler.finishFileUpload(args.location)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except AuthorizationException as aze:
-      msg_type = TMessageType.REPLY
-      result.aze = aze
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("finishFileUpload", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
+  def send_getTopologyInfo(self, id):
+    self._oprot.writeMessageBegin('getTopologyInfo', TMessageType.CALL, self._seqid)
+    args = getTopologyInfo_args()
+    args.id = id
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
 
-  def process_beginFileDownload(self, seqid, iprot, oprot):
-    args = beginFileDownload_args()
-    args.read(iprot)
+  def recv_getTopologyInfo(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = getTopologyInfo_result()
+    result.read(iprot)
     iprot.readMessageEnd()
-    result = beginFileDownload_result()
-    try:
-      result.success = self._handler.beginFileDownload(args.file)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except AuthorizationException as aze:
-      msg_type = TMessageType.REPLY
-      result.aze = aze
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("beginFileDownload", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
+    if result.success is not None:
+      return result.success
+    if result.e is not None:
+      raise result.e
+    if result.aze is not None:
+      raise result.aze
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyInfo failed: unknown result")
 
-  def process_downloadChunk(self, seqid, iprot, oprot):
-    args = downloadChunk_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = downloadChunk_result()
-    try:
-      result.success = self._handler.downloadChunk(args.id)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except AuthorizationException as aze:
-      msg_type = TMessageType.REPLY
-      result.aze = aze
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("downloadChunk", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
+  def getTopologyInfoWithOpts(self, id, options):
+    """
+    Parameters:
+     - id
+     - options
+    """
+    self.send_getTopologyInfoWithOpts(id, options)
+    return self.recv_getTopologyInfoWithOpts()
 
-  def process_getNimbusConf(self, seqid, iprot, oprot):
-    args = getNimbusConf_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = getNimbusConf_result()
-    try:
-      result.success = self._handler.getNimbusConf()
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except AuthorizationException as aze:
-      msg_type = TMessageType.REPLY
-      result.aze = aze
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("getNimbusConf", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
+  def send_getTopologyInfoWithOpts(self, id, options):
+    self._oprot.writeMessageBegin('getTopologyInfoWithOpts', TMessageType.CALL, self._seqid)
+    args = getTopologyInfoWithOpts_args()
+    args.id = id
+    args.options = options
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
 
-  def process_getClusterInfo(self, seqid, iprot, oprot):
-    args = getClusterInfo_args()
-    args.read(iprot)
+  def recv_getTopologyInfoWithOpts(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = getTopologyInfoWithOpts_result()
+    result.read(iprot)
     iprot.readMessageEnd()
-    result = getClusterInfo_result()
-    try:
-      result.success = self._handler.getClusterInfo()
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except AuthorizationException as aze:
-      msg_type = TMessageType.REPLY
-      result.aze = aze
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("getClusterInfo", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
+    if result.success is not None:
+      return result.success
+    if result.e is not None:
+      raise result.e
+    if result.aze is not None:
+      raise result.aze
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyInfoWithOpts failed: unknown result")
 
-  def process_getTopologyInfo(self, seqid, iprot, oprot):
-    args = getTopologyInfo_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = getTopologyInfo_result()
-    try:
-      result.success = self._handler.getTopologyInfo(args.id)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except NotAliveException as e:
-      msg_type = TMessageType.REPLY
-      result.e = e
-    except AuthorizationException as aze:
-      msg_type = TMessageType.REPLY
-      result.aze = aze
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("getTopologyInfo", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
-
-  def process_getTopologyInfoWithOpts(self, seqid, iprot, oprot):
-    args = getTopologyInfoWithOpts_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = getTopologyInfoWithOpts_result()
-    try:
-      result.success = self._handler.getTopologyInfoWithOpts(args.id, args.options)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except NotAliveException as e:
-      msg_type = TMessageType.REPLY
-      result.e = e
-    except AuthorizationException as aze:
-      msg_type = TMessageType.REPLY
-      result.aze = aze
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("getTopologyInfoWithOpts", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
+  def getTopologyPageInfo(self, id, window, is_include_sys):
+    """
+    Parameters:
+     - id
+     - window
+     - is_include_sys
+    """
+    self.send_getTopologyPageInfo(id, window, is_include_sys)
+    return self.recv_getTopologyPageInfo()
 
-  def process_getTopologyPageInfo(self, seqid, iprot, oprot):
+  def send_getTopologyPageInfo(self, id, window, is_include_sys):
+    self._oprot.writeMessageBegin('getTopologyPageInfo', TMessageType.CALL, self._seqid)
     args = getTopologyPageInfo_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
+    args.id = id
+    args.window = window
+    args.is_include_sys = is_include_sys
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_getTopologyPageInfo(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
     result = getTopologyPageInfo_result()
-    try:
-      result.success = self._handler.getTopologyPageInfo(args.id, args.window, args.is_include_sys)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except NotAliveException as e:
-      msg_type = TMessageType.REPLY
-      result.e = e
-    except AuthorizationException as aze:
-      msg_type = TMessageType.REPLY
-      result.aze = aze
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("getTopologyPageInfo", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.e is not None:
+      raise result.e
+    if result.aze is not None:
+      raise result.aze
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyPageInfo failed: unknown result")
 
-  def process_getComponentPageInfo(self, seqid, iprot, oprot):
+  def getComponentPageInfo(self, topology_id, component_id, window, is_include_sys):
+    """
+    Parameters:
+     - topology_id
+     - component_id
+     - window
+     - is_include_sys
+    """
+    self.send_getComponentPageInfo(topology_id, component_id, window, is_include_sys)
+    return self.recv_getComponentPageInfo()
+
+  def send_getComponentPageInfo(self, topology_id, component_id, window, is_include_sys):
+    self._oprot.writeMessageBegin('getComponentPageInfo', TMessageType.CALL, self._seqid)
     args = getComponentPageInfo_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
+    args.topology_id = topology_id
+    args.component_id = component_id
+    args.window = window
+    args.is_include_sys = is_include_sys
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_getComponentPageInfo(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
     result = getComponentPageInfo_result()
-    try:
-      result.success = self._handler.getComponentPageInfo(args.topology_id, args.component_id, args.window, args.is_include_sys)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except NotAliveException as e:
-      msg_type = TMessageType.REPLY
-      result.e = e
-    except AuthorizationException as aze:
-      msg_type = TMessageType.REPLY
-      result.aze = aze
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("getComponentPageInfo", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.e is not None:
+      raise result.e
+    if result.aze is not None:
+      raise result.aze
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getComponentPageInfo failed: unknown result")
 
-  def process_getTopologyConf(self, seqid, iprot, oprot):
+  def getTopologyConf(self, id):
+    """
+    Parameters:
+     - id
+    """
+    self.send_getTopologyConf(id)
+    return self.recv_getTopologyConf()
+
+  def send_getTopologyConf(self, id):
+    self._oprot.writeMessageBegin('getTopologyConf', TMessageType.CALL, self._seqid)
     args = getTopologyConf_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
+    args.id = id
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_getTopologyConf(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
     result = getTopologyConf_result()
-    try:
-      result.success = self._handler.getTopologyConf(args.id)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except NotAliveException as e:
-      msg_type = TMessageType.REPLY
-      result.e = e
-    except AuthorizationException as aze:
-      msg_type = TMessageType.REPLY
-      result.aze = aze
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("getTopologyConf", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.e is not None:
+      raise result.e
+    if result.aze is not None:
+      raise result.aze
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyConf failed: unknown result")
 
-  def process_getTopology(self, seqid, iprot, oprot):
+  def getTopology(self, id):
+    """
+    Returns the compiled topology that contains ackers and metrics consumsers. Compare {@link #getUserTopology(String id)}.
+
+    Parameters:
+     - id
+    """
+    self.send_getTopology(id)
+    return self.recv_getTopology()
+
+  def send_getTopology(self, id):
+    self._oprot.writeMessageBegin('getTopology', TMessageType.CALL, self._seqid)
     args = getTopology_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
+    args.id = id
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_getTopology(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
     result = getTopology_result()
-    try:
-      result.success = self._handler.getTopology(args.id)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except NotAliveException as e:
-      msg_type = TMessageType.REPLY
-      result.e = e
-    except AuthorizationException as aze:
-      msg_type = TMessageType.REPLY
-      result.aze = aze
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("getTopology", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.e is not None:
+      raise result.e
+    if result.aze is not None:
+      raise result.aze
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopology failed: unknown result")
+
+  def getUserTopology(self, id):
+    """
+    Returns the user specified topology as submitted originally. Compare {@link #getTopology(String id)}.
+
+    Parameters:
+     - id
+    """
+    self.send_getUserTopology(id)
+    return self.recv_getUserTopology()
+
+  def send_getUserTopology(self, id):
+    self._oprot.writeMessageBegin('getUserTopology', TMessageType.CALL, self._seqid)
+    args = getUserTopology_args()
+    args.id = id
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_getUserTopology(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = getUserTopology_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.e is not None:
+      raise result.e
+    if result.aze is not None:
+      raise result.aze
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getUserTopology failed: unknown result")
+
+  def getTopologyHistory(self, user):
+    """
+    Parameters:
+     - user
+    """
+    self.send_getTopologyHistory(user)
+    return self.recv_getTopologyHistory()
+
+  def send_getTopologyHistory(self, user):
+    self._oprot.writeMessageBegin('getTopologyHistory', TMessageType.CALL, self._seqid)
+    args = getTopologyHistory_args()
+    args.user = user
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_getTopologyHistory(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = getTopologyHistory_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.aze is not None:
+      raise result.aze
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyHistory failed: unknown result")
+
+
+class Processor(Iface, TProcessor):
+  def __init__(self, handler):
+    self._handler = handler
+    self._processMap = {}
+    self._processMap["submitTopology"] = Processor.process_submitTopology
+    self._processMap["submitTopologyWithOpts"] = Processor.process_submitTopologyWithOpts
+    self._processMap["killTopology"] = Processor.process_killTopology
+    self._processMap["killTopologyWithOpts"] = Processor.process_killTopologyWithOpts
+    self._processMap["activate"] = Processor.process_activate
+    self._processMap["deactivate"] = Processor.process_deactivate
+    self._processMap["rebalance"] = Processor.process_rebalance
+    self._processMap["setLogConfig"] = Processor.process_setLogConfig
+    self._processMap["getLogConfig"] = Processor.process_getLogConfig
+    self._processMap["debug"] = Processor.process_debug
+    self._processMap["setWorkerProfiler"] = Processor.process_setWorkerProfiler
+    self._processMap["getComponentPendingProfileActions"] = Processor.process_getComponentPendingProfileActions
+    self._processMap["uploadNewCredentials"] = Processor.process_uploadNewCredentials
+    self._processMap["beginCreateBlob"] = Processor.process_beginCreateBlob
+    self._processMap["beginUpdateBlob"] = Processor.process_beginUpdateBlob
+    self._processMap["uploadBlobChunk"] = Processor.process_uploadBlobChunk
+    self._processMap["finishBlobUpload"] = Processor.process_finishBlobUpload
+    self._processMap["cancelBlobUpload"] = Processor.process_cancelBlobUpload
+    self._processMap["getBlobMeta"] = Processor.process_getBlobMeta
+    self._processMap["setBlobMeta"] = Processor.process_setBlobMeta
+    self._processMap["beginBlobDownload"] = Processor.process_beginBlobDownload
+    self._processMap["downloadBlobChunk"] = Processor.process_downloadBlobChunk
+    self._processMap["deleteBlob"] = Processor.process_deleteBlob
+    self._processMap["listBlobs"] = Processor.process_listBlobs
+    self._processMap["getBlobReplication"] = Processor.process_getBlobReplication
+    self._processMap["updateBlobReplication"] = Processor.process_updateBlobReplication
+    self._processMap["createStateInZookeeper"] = Processor.process_createStateInZookeeper
+    self._processMap["beginFileUpload"] = Processor.process_beginFileUpload
+    self._processMap["uploadChunk"] = Processor.process_uploadChunk
+    self._processMap["finishFileUpload"] = Processor.process_finishFileUpload
+    self._processMap["beginFileDownload"] = Processor.process_beginFileDownload
+    self._processMap["downloadChunk"] = Processor.process_downloadChunk
+    self._processMap["getNimbusConf"] = Processor.process_getNimbusConf
+    self._processMap["getClusterInfo"] = Processor.process_getClusterInfo
+    self._processMap["getTopologyInfo"] = Processor.process_getTopologyInfo
+    self._processMap["getTopologyInfoWithOpts"] = Processor.process_getTopologyInfoWithOpts
+    self._processMap["getTopologyPageInfo"] = Processor.process_getTopologyPageInfo
+    self._processMap["getComponentPageInfo"] = Processor.process_getComponentPageInfo
+    self._processMap["getTopologyConf"] = Processor.process_getTopologyConf
+    self._processMap["getTopology"] = Processor.process_getTopology
+    self._processMap["getUserTopology"] = Processor.process_getUserTopology
+    self._processMap["getTopologyHistory"] = Processor.process_getTopologyHistory
+
+  def process(self, iprot, oprot):
+    (name, type, seqid) = iprot.readMessageBegin()
+    if name not in self._processMap:
+      iprot.skip(TType.STRUCT)
+      iprot.readMessageEnd()
+      x = TApplicationException(TApplicationException.UNKNOWN_METHOD, 'Unknown function %s' % (name))
+      oprot.writeMessageBegin(name, TMessageType.EXCEPTION, seqid)
+      x.write(oprot)
+      oprot.writeMessageEnd()
+      oprot.trans.flush()
+      return
+    else:
+      self._processMap[name](self, seqid, iprot, oprot)
+    return True
+
+  def process_submitTopology(self, seqid, iprot, oprot):
+    args = submitTopology_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = submitTopology_result()
+    try:
+      self._handler.submitTopology(args.name, args.uploadedJarLocation, args.jsonConf, args.topology)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AlreadyAliveException as e:
+      msg_type = TMessageType.REPLY
+      result.e = e
+    except InvalidTopologyException as ite:
+      msg_type = TMessageType.REPLY
+      result.ite = ite
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
+      result.aze = aze
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("submitTopology", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_submitTopologyWithOpts(self, seqid, iprot, oprot):
+    args = submitTopologyWithOpts_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = submitTopologyWithOpts_result()
+    try:
+      self._handler.submitTopologyWithOpts(args.name, args.uploadedJarLocation, args.jsonConf, args.topology, args.options)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AlreadyAliveException as e:
+      msg_type = TMessageType.REPLY
+      result.e = e
+    except InvalidTopologyException as ite:
+      msg_type = TMessageType.REPLY
+      result.ite = ite
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
+      result.aze = aze
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("submitTopologyWithOpts", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_killTopology(self, seqid, iprot, oprot):
+    args = killTopology_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = killTopology_result()
+    try:
+      self._handler.killTopology(args.name)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NotAliveException as e:
+      msg_type = TMessageType.REPLY
+      result.e = e
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
+      result.aze = aze
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("killTopology", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_killTopologyWithOpts(self, seqid, iprot, oprot):
+    args = killTopologyWithOpts_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = killTopologyWithOpts_result()
+    try:
+      self._handler.killTopologyWithOpts(args.name, args.options)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NotAliveException as e:
+      msg_type = TMessageType.REPLY
+      result.e = e
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
+      result.aze = aze
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("killTopologyWithOpts", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
     oprot.trans.flush()
 
-  def process_getUserTopology(self, seqid, iprot, oprot):
-    args = getUserTopology_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = getUserTopology_result()
-    try:
-      result.success = self._handler.getUserTopology(args.id)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except NotAliveException as e:
-      msg_type = TMessageType.REPLY
-      result.e = e
-    except AuthorizationException as aze:
-      msg_type = TMessageType.REPLY
-      result.aze = aze
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("getUserTopology", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
+  def process_activate(self, seqid, iprot, oprot):
+    args = activate_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = activate_result()
+    try:
+      self._handler.activate(args.name)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NotAliveException as e:
+      msg_type = TMessageType.REPLY
+      result.e = e
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
+      result.aze = aze
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("activate", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_deactivate(self, seqid, iprot, oprot):
+    args = deactivate_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = deactivate_result()
+    try:
+      self._handler.deactivate(args.name)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NotAliveException as e:
+      msg_type = TMessageType.REPLY
+      result.e = e
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
+      result.aze = aze
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("deactivate", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_rebalance(self, seqid, iprot, oprot):
+    args = rebalance_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = rebalance_result()
+    try:
+      self._handler.rebalance(args.name, args.options)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NotAliveException as e:
+      msg_type = TMessageType.REPLY
+      result.e = e
+    except InvalidTopologyException as ite:
+      msg_type = TMessageType.REPLY
+      result.ite = ite
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
+      result.aze = aze
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("rebalance", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_setLogConfig(self, seqid, iprot, oprot):
+    args = setLogConfig_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = setLogConfig_result()
+    try:
+      self._handler.setLogConfig(args.name, args.config)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("setLogConfig", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_getLogConfig(self, seqid, iprot, oprot):
+    args = getLogConfig_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = getLogConfig_result()
+    try:
+      result.success = self._handler.getLogConfig(args.name)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("getLogConfig", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_debug(self, seqid, iprot, oprot):
+    args = debug_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = debug_result()
+    try:
+      self._handler.debug(args.name, args.component, args.enable, args.samplingPercentage)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NotAliveException as e:
+      msg_type = TMessageType.REPLY
+      result.e = e
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
+      result.aze = aze
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("debug", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_setWorkerProfiler(self, seqid, iprot, oprot):
+    args = setWorkerProfiler_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = setWorkerProfiler_result()
+    try:
+      self._handler.setWorkerProfiler(args.id, args.profileRequest)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("setWorkerProfiler", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_getComponentPendingProfileActions(self, seqid, iprot, oprot):
+    args = getComponentPendingProfileActions_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = getComponentPendingProfileActions_result()
+    try:
+      result.success = self._handler.getComponentPendingProfileActions(args.id, args.component_id, args.action)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("getComponentPendingProfileActions", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_uploadNewCredentials(self, seqid, iprot, oprot):
+    args = uploadNewCredentials_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = uploadNewCredentials_result()
+    try:
+      self._handler.uploadNewCredentials(args.name, args.creds)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NotAliveException as e:
+      msg_type = TMessageType.REPLY
+      result.e = e
+    except InvalidTopologyException as ite:
+      msg_type = TMessageType.REPLY
+      result.ite = ite
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
+      result.aze = aze
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("uploadNewCredentials", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_beginCreateBlob(self, seqid, iprot, oprot):
+    args = beginCreateBlob_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = beginCreateBlob_result()
+    try:
+      result.success = self._handler.beginCreateBlob(args.key, args.meta)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
+      result.aze = aze
+    except KeyAlreadyExistsException as kae:
+      msg_type = TMessageType.REPLY
+      result.kae = kae
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("beginCreateBlob", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_beginUpdateBlob(self, seqid, iprot, oprot):
+    args = beginUpdateBlob_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = beginUpdateBlob_result()
+    try:
+      result.success = self._handler.beginUpdateBlob(args.key)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
+      result.aze = aze
+    except KeyNotFoundException as knf:
+      msg_type = TMessageType.REPLY
+      result.knf = knf
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("beginUpdateBlob", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_uploadBlobChunk(self, seqid, iprot, oprot):
+    args = uploadBlobChunk_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = uploadBlobChunk_result()
+    try:
+      self._handler.uploadBlobChunk(args.session, args.chunk)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
+      result.aze = aze
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("uploadBlobChunk", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_finishBlobUpload(self, seqid, iprot, oprot):
+    args = finishBlobUpload_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = finishBlobUpload_result()
+    try:
+      self._handler.finishBlobUpload(args.session)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
+      result.aze = aze
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("finishBlobUpload", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_cancelBlobUpload(self, seqid, iprot, oprot):
+    args = cancelBlobUpload_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = cancelBlobUpload_result()
+    try:
+      self._handler.cancelBlobUpload(args.session)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
+      result.aze = aze
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("cancelBlobUpload", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_getBlobMeta(self, seqid, iprot, oprot):
+    args = getBlobMeta_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = getBlobMeta_result()
+    try:
+      result.success = self._handler.getBlobMeta(args.key)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
+      result.aze = aze
+    except KeyNotFoundException as knf:
+      msg_type = TMessageType.REPLY
+      result.knf = knf
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("getBlobMeta", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_setBlobMeta(self, seqid, iprot, oprot):
+    args = setBlobMeta_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = setBlobMeta_result()
+    try:
+      self._handler.setBlobMeta(args.key, args.meta)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
+      result.aze = aze
+    except KeyNotFoundException as knf:
+      msg_type = TMessageType.REPLY
+      result.knf = knf
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("setBlobMeta", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_beginBlobDownload(self, seqid, iprot, oprot):
+    args = beginBlobDownload_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = beginBlobDownload_result()
+    try:
+      result.success = self._handler.beginBlobDownload(args.key)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AuthorizationException as aze:
+      msg

<TRUNCATED>

[14/17] storm git commit: client blobstore interface documentation update

Posted by bo...@apache.org.
client blobstore interface documentation update


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

Branch: refs/heads/master
Commit: 6687bedbe86c1b04cecf18e02e6e2359ce2c3ead
Parents: 57240b4
Author: Sanket <sc...@untilservice-lm>
Authored: Mon Nov 30 18:02:39 2015 -0600
Committer: Sanket <sc...@untilservice-lm>
Committed: Mon Nov 30 18:02:39 2015 -0600

----------------------------------------------------------------------
 .../storm/hdfs/blobstore/HdfsBlobStore.java     |  14 +++
 .../hdfs/blobstore/HdfsClientBlobStore.java     |   5 +
 .../storm/blobstore/ClientBlobStore.java        | 126 ++++++++++++++++++-
 .../storm/blobstore/LocalFsBlobStore.java       |  15 +++
 .../storm/blobstore/NimbusBlobStore.java        |   8 ++
 5 files changed, 166 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/6687bedb/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java
----------------------------------------------------------------------
diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java
index 144ad71..c344bcf 100644
--- a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java
+++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsBlobStore.java
@@ -54,6 +54,20 @@ 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.
+ *
+ * We currently have NIMBUS_ADMINS and SUPERVISOR_ADMINS configuration. NIMBUS_ADMINS are given READ, WRITE and ADMIN
+ * access whereas the SUPERVISOR_ADMINS are given READ access in order to read and download the blobs form the nimbus.
+ *
+ * The ACLs for the blob store are validated against whether the subject is a NIMBUS_ADMIN, SUPERVISOR_ADMIN or USER
+ * who has read, write or admin privileges in order to perform respective operations on the blob.
+ *
+ * For hdfs blob store
+ * 1. The USER interacts with nimbus to upload and access blobs through NimbusBlobStore Client API. Here, unlike
+ * local blob store which stores the blobs locally, the nimbus talks to HDFS to upload the blobs.
+ * 2. The USER sets the ACLs, and the blob access is validated against these ACLs.
+ * 3. The SUPERVISOR interacts with nimbus thorugh HdfsClientBlobStore to download the blobs. Here, unlike local
+ * blob store the supervisor interacts with HDFS directly to download the blobs. The call to HdfsBlobStore is made as a "null"
+ * subject. The blobstore gets the hadoop user and validates permissions for the supervisor.
  */
 public class HdfsBlobStore extends BlobStore {
     public static final Logger LOG = LoggerFactory.getLogger(HdfsBlobStore.class);

http://git-wip-us.apache.org/repos/asf/storm/blob/6687bedb/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsClientBlobStore.java
----------------------------------------------------------------------
diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsClientBlobStore.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsClientBlobStore.java
index ec17dae..18c97da 100644
--- a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsClientBlobStore.java
+++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/blobstore/HdfsClientBlobStore.java
@@ -35,6 +35,11 @@ import java.util.Map;
 /**
  *  Client to access the HDFS blobStore. At this point, this is meant to only be used by the
  *  supervisor.  Don't trust who the client says they are so pass null for all Subjects.
+ *
+ *  The HdfsBlobStore implementation takes care of the null Subjects. It assigns Subjects
+ *  based on what hadoop says who the users are. These users must be configured accordingly
+ *  in the SUPERVISOR_ADMINS for ACL validation and for the supervisors to download the blobs.
+ *  This API is only used by the supervisor in order to talk directly to HDFS.
  */
 public class HdfsClientBlobStore extends ClientBlobStore {
     private static final Logger LOG = LoggerFactory.getLogger(HdfsClientBlobStore.class);

http://git-wip-us.apache.org/repos/asf/storm/blob/6687bedb/storm-core/src/jvm/backtype/storm/blobstore/ClientBlobStore.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/blobstore/ClientBlobStore.java b/storm-core/src/jvm/backtype/storm/blobstore/ClientBlobStore.java
index cc40aff..6408469 100644
--- a/storm-core/src/jvm/backtype/storm/blobstore/ClientBlobStore.java
+++ b/storm-core/src/jvm/backtype/storm/blobstore/ClientBlobStore.java
@@ -28,22 +28,137 @@ import backtype.storm.utils.NimbusClient;
 import java.util.Iterator;
 import java.util.Map;
 
+/**
+ * The ClientBlobStore has two concrete implementations
+ * 1. NimbusBlobStore
+ * 2. HdfsClientBlobStore
+ *
+ * Create, update, read and delete are some of the basic operations defined by this interface.
+ * Each operation is validated for permissions against an user. We currently have NIMBUS_ADMINS and SUPERVISOR_ADMINS
+ * configuration. NIMBUS_ADMINS are given READ, WRITE and ADMIN access whereas the SUPERVISOR_ADMINS are given READ
+ * access in order to read and download the blobs form the nimbus.
+ *
+ * The ACLs for the blob store are validated against whether the subject is a NIMBUS_ADMIN, SUPERVISOR_ADMIN or USER
+ * who has read, write or admin privileges in order to perform respective operations on the blob.
+ *
+ * For more detailed implementation
+ * @see backtype.storm.blobstore.NimbusBlobStore
+ * @see backtype.storm.blobstore.LocalFsBlobStore
+ * @see org.apache.storm.hdfs.blobstore.HdfsClientBlobStore
+ * @see org.apache.storm.hdfs.blobstore.HdfsBlobStore
+ */
 public abstract class ClientBlobStore implements Shutdownable {
     protected Map conf;
 
+    /**
+     * Sets up the client API by parsing the configs.
+     * @param conf The storm conf containing the config details.
+     */
     public abstract void prepare(Map conf);
+
+    /**
+     * Client facing API to create a blob.
+     * @param key blob key name.
+     * @param meta contains ACL information.
+     * @return AtomicOutputStream returns an output stream into which data can be written.
+     * @throws AuthorizationException
+     * @throws KeyAlreadyExistsException
+     */
     protected abstract AtomicOutputStream createBlobToExtend(String key, SettableBlobMeta meta) throws AuthorizationException, KeyAlreadyExistsException;
+
+    /**
+     * Client facing API to update a blob.
+     * @param key blob key name.
+     * @return AtomicOutputStream returns an output stream into which data can be written.
+     * @throws AuthorizationException
+     * @throws KeyNotFoundException
+     */
     public abstract AtomicOutputStream updateBlob(String key) throws AuthorizationException, KeyNotFoundException;
+
+    /**
+     * Client facing API to read the metadata information.
+     * @param key blob key name.
+     * @return AtomicOutputStream returns an output stream into which data can be written.
+     * @throws AuthorizationException
+     * @throws KeyNotFoundException
+     */
     public abstract ReadableBlobMeta getBlobMeta(String key) throws AuthorizationException, KeyNotFoundException;
+
+    /**
+     * Client facing API to set the metadata for a blob.
+     * @param key blob key name.
+     * @param meta contains ACL information.
+     * @throws AuthorizationException
+     * @throws KeyNotFoundException
+     */
     protected abstract void setBlobMetaToExtend(String key, SettableBlobMeta meta) throws AuthorizationException, KeyNotFoundException;
+
+    /**
+     * Client facing API to delete a blob.
+     * @param key blob key name.
+     * @throws AuthorizationException
+     * @throws KeyNotFoundException
+     */
     public abstract void deleteBlob(String key) throws AuthorizationException, KeyNotFoundException;
+
+    /**
+     * Client facing API to read a blob.
+     * @param key blob key name.
+     * @return an InputStream to read the metadata for a blob.
+     * @throws AuthorizationException
+     * @throws KeyNotFoundException
+     */
     public abstract InputStreamWithMeta getBlob(String key) throws AuthorizationException, KeyNotFoundException;
+
+    /**
+     * @return Iterator for a list of keys currently present in the blob store.
+     */
     public abstract Iterator<String> listKeys();
-    public abstract int getBlobReplication(String Key) throws AuthorizationException, KeyNotFoundException;
-    public abstract int updateBlobReplication(String Key, int replication) throws AuthorizationException, KeyNotFoundException;
+
+    /**
+     * Client facing API to read the replication of a blob.
+     * @param key blob key name.
+     * @return int indicates the replication factor of a blob.
+     * @throws AuthorizationException
+     * @throws KeyNotFoundException
+     */
+    public abstract int getBlobReplication(String key) throws AuthorizationException, KeyNotFoundException;
+
+    /**
+     * Client facing API to update the replication of a blob.
+     * @param key blob key name.
+     * @param replication int indicates the replication factor a blob has to be set.
+     * @return int indicates the replication factor of a blob.
+     * @throws AuthorizationException
+     * @throws KeyNotFoundException
+     */
+    public abstract int updateBlobReplication(String key, int replication) throws AuthorizationException, KeyNotFoundException;
+
+    /**
+     * Client facing API to set a nimbus client.
+     * @param conf storm conf
+     * @param client NimbusClient
+     * @return indicates where the client connection has been setup.
+     */
     public abstract boolean setClient(Map conf, NimbusClient client);
+
+    /**
+     * Creates state inside a zookeeper.
+     * Required for blobstore to write to zookeeper
+     * when Nimbus HA is turned on in order to maintain
+     * state consistency
+     * @param key
+     */
     public abstract void createStateInZookeeper(String key);
 
+    /**
+     * Client facing API to create a blob.
+     * @param key blob key name.
+     * @param meta contains ACL information.
+     * @return AtomicOutputStream returns an output stream into which data can be written.
+     * @throws AuthorizationException
+     * @throws KeyAlreadyExistsException
+     */
     public final AtomicOutputStream createBlob(String key, SettableBlobMeta meta) throws AuthorizationException, KeyAlreadyExistsException {
         if (meta !=null && meta.is_set_acl()) {
             BlobStoreAclHandler.validateSettableACLs(key, meta.get_acl());
@@ -51,6 +166,13 @@ public abstract class ClientBlobStore implements Shutdownable {
         return createBlobToExtend(key, meta);
     }
 
+    /**
+     * Client facing API to set the metadata for a blob.
+     * @param key blob key name.
+     * @param meta contains ACL information.
+     * @throws AuthorizationException
+     * @throws KeyNotFoundException
+     */
     public final void setBlobMeta(String key, SettableBlobMeta meta) throws AuthorizationException, KeyNotFoundException {
         if (meta !=null && meta.is_set_acl()) {
             BlobStoreAclHandler.validateSettableACLs(key, meta.get_acl());

http://git-wip-us.apache.org/repos/asf/storm/blob/6687bedb/storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStore.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStore.java b/storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStore.java
index 0941b9a..ac7a4bd 100644
--- a/storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStore.java
+++ b/storm-core/src/jvm/backtype/storm/blobstore/LocalFsBlobStore.java
@@ -48,6 +48,21 @@ import static backtype.storm.blobstore.BlobStoreAclHandler.WRITE;
 
 /**
  * Provides a local file system backed blob store implementation for Nimbus.
+ *
+ * For a local blob store the user and the supervisor use NimbusBlobStore Client API in order to talk to nimbus through thrift.
+ * The authentication and authorization here is based on the subject.
+ * We currently have NIMBUS_ADMINS and SUPERVISOR_ADMINS configuration. NIMBUS_ADMINS are given READ, WRITE and ADMIN
+ * access whereas the SUPERVISOR_ADMINS are given READ access in order to read and download the blobs form the nimbus.
+ *
+ * The ACLs for the blob store are validated against whether the subject is a NIMBUS_ADMIN, SUPERVISOR_ADMIN or USER
+ * who has read, write or admin privileges in order to perform respective operations on the blob.
+ *
+ * For local blob store
+ * 1. The USER interacts with nimbus to upload and access blobs through NimbusBlobStore Client API.
+ * 2. The USER sets the ACLs, and the blob access is validated against these ACLs.
+ * 3. The SUPERVISOR interacts with nimbus through the NimbusBlobStore Client API to download the blobs.
+ * The supervisors principal should match the set of users configured into SUPERVISOR_ADMINS.
+ * Here, the PrincipalToLocalPlugin takes care of mapping the principal to user name before the ACL validation.
  */
 public class LocalFsBlobStore extends BlobStore {
     public static final Logger LOG = LoggerFactory.getLogger(LocalFsBlobStore.class);

http://git-wip-us.apache.org/repos/asf/storm/blob/6687bedb/storm-core/src/jvm/backtype/storm/blobstore/NimbusBlobStore.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/blobstore/NimbusBlobStore.java b/storm-core/src/jvm/backtype/storm/blobstore/NimbusBlobStore.java
index bf084bb..334e6bb 100644
--- a/storm-core/src/jvm/backtype/storm/blobstore/NimbusBlobStore.java
+++ b/storm-core/src/jvm/backtype/storm/blobstore/NimbusBlobStore.java
@@ -37,6 +37,14 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.NoSuchElementException;
 
+/**
+ * NimbusBlobStore is a USER facing client API to perform
+ * basic operations such as create, update, delete and read
+ * for local and hdfs blob store.
+ *
+ * For local blob store it is also the client facing API for
+ * supervisor in order to download blobs from nimbus.
+ */
 public class NimbusBlobStore extends ClientBlobStore {
     private static final Logger LOG = LoggerFactory.getLogger(NimbusBlobStore.class);
 


[05/17] storm git commit: Blobstore API STORM- 876

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java b/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java
index 5ef4b5b..875c2ba 100644
--- a/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java
@@ -461,13 +461,13 @@ public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fie
           case 2: // PORT
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set524 = iprot.readSetBegin();
-                struct.port = new HashSet<Long>(2*_set524.size);
-                long _elem525;
-                for (int _i526 = 0; _i526 < _set524.size; ++_i526)
+                org.apache.thrift.protocol.TSet _set540 = iprot.readSetBegin();
+                struct.port = new HashSet<Long>(2*_set540.size);
+                long _elem541;
+                for (int _i542 = 0; _i542 < _set540.size; ++_i542)
                 {
-                  _elem525 = iprot.readI64();
-                  struct.port.add(_elem525);
+                  _elem541 = iprot.readI64();
+                  struct.port.add(_elem541);
                 }
                 iprot.readSetEnd();
               }
@@ -498,9 +498,9 @@ public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fie
         oprot.writeFieldBegin(PORT_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.port.size()));
-          for (long _iter527 : struct.port)
+          for (long _iter543 : struct.port)
           {
-            oprot.writeI64(_iter527);
+            oprot.writeI64(_iter543);
           }
           oprot.writeSetEnd();
         }
@@ -526,9 +526,9 @@ public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fie
       oprot.writeString(struct.node);
       {
         oprot.writeI32(struct.port.size());
-        for (long _iter528 : struct.port)
+        for (long _iter544 : struct.port)
         {
-          oprot.writeI64(_iter528);
+          oprot.writeI64(_iter544);
         }
       }
     }
@@ -539,13 +539,13 @@ public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fie
       struct.node = iprot.readString();
       struct.set_node_isSet(true);
       {
-        org.apache.thrift.protocol.TSet _set529 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.port = new HashSet<Long>(2*_set529.size);
-        long _elem530;
-        for (int _i531 = 0; _i531 < _set529.size; ++_i531)
+        org.apache.thrift.protocol.TSet _set545 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.port = new HashSet<Long>(2*_set545.size);
+        long _elem546;
+        for (int _i547 = 0; _i547 < _set545.size; ++_i547)
         {
-          _elem530 = iprot.readI64();
-          struct.port.add(_elem530);
+          _elem546 = iprot.readI64();
+          struct.port.add(_elem546);
         }
       }
       struct.set_port_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/generated/ReadableBlobMeta.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ReadableBlobMeta.java b/storm-core/src/jvm/backtype/storm/generated/ReadableBlobMeta.java
new file mode 100644
index 0000000..69355a5
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/ReadableBlobMeta.java
@@ -0,0 +1,510 @@
+/**
+ * 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.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class ReadableBlobMeta implements org.apache.thrift.TBase<ReadableBlobMeta, ReadableBlobMeta._Fields>, java.io.Serializable, Cloneable, Comparable<ReadableBlobMeta> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ReadableBlobMeta");
+
+  private static final org.apache.thrift.protocol.TField SETTABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("settable", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+  private static final org.apache.thrift.protocol.TField VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("version", org.apache.thrift.protocol.TType.I64, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ReadableBlobMetaStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ReadableBlobMetaTupleSchemeFactory());
+  }
+
+  private SettableBlobMeta settable; // required
+  private long version; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    SETTABLE((short)1, "settable"),
+    VERSION((short)2, "version");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // SETTABLE
+          return SETTABLE;
+        case 2: // VERSION
+          return VERSION;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __VERSION_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.SETTABLE, new org.apache.thrift.meta_data.FieldMetaData("settable", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SettableBlobMeta.class)));
+    tmpMap.put(_Fields.VERSION, new org.apache.thrift.meta_data.FieldMetaData("version", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ReadableBlobMeta.class, metaDataMap);
+  }
+
+  public ReadableBlobMeta() {
+  }
+
+  public ReadableBlobMeta(
+    SettableBlobMeta settable,
+    long version)
+  {
+    this();
+    this.settable = settable;
+    this.version = version;
+    set_version_isSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ReadableBlobMeta(ReadableBlobMeta other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_settable()) {
+      this.settable = new SettableBlobMeta(other.settable);
+    }
+    this.version = other.version;
+  }
+
+  public ReadableBlobMeta deepCopy() {
+    return new ReadableBlobMeta(this);
+  }
+
+  @Override
+  public void clear() {
+    this.settable = null;
+    set_version_isSet(false);
+    this.version = 0;
+  }
+
+  public SettableBlobMeta get_settable() {
+    return this.settable;
+  }
+
+  public void set_settable(SettableBlobMeta settable) {
+    this.settable = settable;
+  }
+
+  public void unset_settable() {
+    this.settable = null;
+  }
+
+  /** Returns true if field settable is set (has been assigned a value) and false otherwise */
+  public boolean is_set_settable() {
+    return this.settable != null;
+  }
+
+  public void set_settable_isSet(boolean value) {
+    if (!value) {
+      this.settable = null;
+    }
+  }
+
+  public long get_version() {
+    return this.version;
+  }
+
+  public void set_version(long version) {
+    this.version = version;
+    set_version_isSet(true);
+  }
+
+  public void unset_version() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VERSION_ISSET_ID);
+  }
+
+  /** Returns true if field version is set (has been assigned a value) and false otherwise */
+  public boolean is_set_version() {
+    return EncodingUtils.testBit(__isset_bitfield, __VERSION_ISSET_ID);
+  }
+
+  public void set_version_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VERSION_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case SETTABLE:
+      if (value == null) {
+        unset_settable();
+      } else {
+        set_settable((SettableBlobMeta)value);
+      }
+      break;
+
+    case VERSION:
+      if (value == null) {
+        unset_version();
+      } else {
+        set_version((Long)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case SETTABLE:
+      return get_settable();
+
+    case VERSION:
+      return get_version();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case SETTABLE:
+      return is_set_settable();
+    case VERSION:
+      return is_set_version();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ReadableBlobMeta)
+      return this.equals((ReadableBlobMeta)that);
+    return false;
+  }
+
+  public boolean equals(ReadableBlobMeta that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_settable = true && this.is_set_settable();
+    boolean that_present_settable = true && that.is_set_settable();
+    if (this_present_settable || that_present_settable) {
+      if (!(this_present_settable && that_present_settable))
+        return false;
+      if (!this.settable.equals(that.settable))
+        return false;
+    }
+
+    boolean this_present_version = true;
+    boolean that_present_version = true;
+    if (this_present_version || that_present_version) {
+      if (!(this_present_version && that_present_version))
+        return false;
+      if (this.version != that.version)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_settable = true && (is_set_settable());
+    list.add(present_settable);
+    if (present_settable)
+      list.add(settable);
+
+    boolean present_version = true;
+    list.add(present_version);
+    if (present_version)
+      list.add(version);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ReadableBlobMeta other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_settable()).compareTo(other.is_set_settable());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_settable()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.settable, other.settable);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_version()).compareTo(other.is_set_version());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_version()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.version, other.version);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ReadableBlobMeta(");
+    boolean first = true;
+
+    sb.append("settable:");
+    if (this.settable == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.settable);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("version:");
+    sb.append(this.version);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_settable()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'settable' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_version()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'version' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+    if (settable != null) {
+      settable.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ReadableBlobMetaStandardSchemeFactory implements SchemeFactory {
+    public ReadableBlobMetaStandardScheme getScheme() {
+      return new ReadableBlobMetaStandardScheme();
+    }
+  }
+
+  private static class ReadableBlobMetaStandardScheme extends StandardScheme<ReadableBlobMeta> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ReadableBlobMeta struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // SETTABLE
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.settable = new SettableBlobMeta();
+              struct.settable.read(iprot);
+              struct.set_settable_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // VERSION
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.version = iprot.readI64();
+              struct.set_version_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ReadableBlobMeta struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.settable != null) {
+        oprot.writeFieldBegin(SETTABLE_FIELD_DESC);
+        struct.settable.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(VERSION_FIELD_DESC);
+      oprot.writeI64(struct.version);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ReadableBlobMetaTupleSchemeFactory implements SchemeFactory {
+    public ReadableBlobMetaTupleScheme getScheme() {
+      return new ReadableBlobMetaTupleScheme();
+    }
+  }
+
+  private static class ReadableBlobMetaTupleScheme extends TupleScheme<ReadableBlobMeta> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ReadableBlobMeta struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      struct.settable.write(oprot);
+      oprot.writeI64(struct.version);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ReadableBlobMeta struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.settable = new SettableBlobMeta();
+      struct.settable.read(iprot);
+      struct.set_settable_isSet(true);
+      struct.version = iprot.readI64();
+      struct.set_version_isSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/generated/SettableBlobMeta.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/SettableBlobMeta.java b/storm-core/src/jvm/backtype/storm/generated/SettableBlobMeta.java
new file mode 100644
index 0000000..b3dd9d8
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/SettableBlobMeta.java
@@ -0,0 +1,567 @@
+/**
+ * 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.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class SettableBlobMeta implements org.apache.thrift.TBase<SettableBlobMeta, SettableBlobMeta._Fields>, java.io.Serializable, Cloneable, Comparable<SettableBlobMeta> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SettableBlobMeta");
+
+  private static final org.apache.thrift.protocol.TField ACL_FIELD_DESC = new org.apache.thrift.protocol.TField("acl", org.apache.thrift.protocol.TType.LIST, (short)1);
+  private static final org.apache.thrift.protocol.TField REPLICATION_FACTOR_FIELD_DESC = new org.apache.thrift.protocol.TField("replication_factor", org.apache.thrift.protocol.TType.I32, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new SettableBlobMetaStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new SettableBlobMetaTupleSchemeFactory());
+  }
+
+  private List<AccessControl> acl; // required
+  private int replication_factor; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    ACL((short)1, "acl"),
+    REPLICATION_FACTOR((short)2, "replication_factor");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // ACL
+          return ACL;
+        case 2: // REPLICATION_FACTOR
+          return REPLICATION_FACTOR;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __REPLICATION_FACTOR_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.REPLICATION_FACTOR};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.ACL, new org.apache.thrift.meta_data.FieldMetaData("acl", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, AccessControl.class))));
+    tmpMap.put(_Fields.REPLICATION_FACTOR, new org.apache.thrift.meta_data.FieldMetaData("replication_factor", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SettableBlobMeta.class, metaDataMap);
+  }
+
+  public SettableBlobMeta() {
+  }
+
+  public SettableBlobMeta(
+    List<AccessControl> acl)
+  {
+    this();
+    this.acl = acl;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public SettableBlobMeta(SettableBlobMeta other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_acl()) {
+      List<AccessControl> __this__acl = new ArrayList<AccessControl>(other.acl.size());
+      for (AccessControl other_element : other.acl) {
+        __this__acl.add(new AccessControl(other_element));
+      }
+      this.acl = __this__acl;
+    }
+    this.replication_factor = other.replication_factor;
+  }
+
+  public SettableBlobMeta deepCopy() {
+    return new SettableBlobMeta(this);
+  }
+
+  @Override
+  public void clear() {
+    this.acl = null;
+    set_replication_factor_isSet(false);
+    this.replication_factor = 0;
+  }
+
+  public int get_acl_size() {
+    return (this.acl == null) ? 0 : this.acl.size();
+  }
+
+  public java.util.Iterator<AccessControl> get_acl_iterator() {
+    return (this.acl == null) ? null : this.acl.iterator();
+  }
+
+  public void add_to_acl(AccessControl elem) {
+    if (this.acl == null) {
+      this.acl = new ArrayList<AccessControl>();
+    }
+    this.acl.add(elem);
+  }
+
+  public List<AccessControl> get_acl() {
+    return this.acl;
+  }
+
+  public void set_acl(List<AccessControl> acl) {
+    this.acl = acl;
+  }
+
+  public void unset_acl() {
+    this.acl = null;
+  }
+
+  /** Returns true if field acl is set (has been assigned a value) and false otherwise */
+  public boolean is_set_acl() {
+    return this.acl != null;
+  }
+
+  public void set_acl_isSet(boolean value) {
+    if (!value) {
+      this.acl = null;
+    }
+  }
+
+  public int get_replication_factor() {
+    return this.replication_factor;
+  }
+
+  public void set_replication_factor(int replication_factor) {
+    this.replication_factor = replication_factor;
+    set_replication_factor_isSet(true);
+  }
+
+  public void unset_replication_factor() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REPLICATION_FACTOR_ISSET_ID);
+  }
+
+  /** Returns true if field replication_factor is set (has been assigned a value) and false otherwise */
+  public boolean is_set_replication_factor() {
+    return EncodingUtils.testBit(__isset_bitfield, __REPLICATION_FACTOR_ISSET_ID);
+  }
+
+  public void set_replication_factor_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REPLICATION_FACTOR_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case ACL:
+      if (value == null) {
+        unset_acl();
+      } else {
+        set_acl((List<AccessControl>)value);
+      }
+      break;
+
+    case REPLICATION_FACTOR:
+      if (value == null) {
+        unset_replication_factor();
+      } else {
+        set_replication_factor((Integer)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case ACL:
+      return get_acl();
+
+    case REPLICATION_FACTOR:
+      return get_replication_factor();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case ACL:
+      return is_set_acl();
+    case REPLICATION_FACTOR:
+      return is_set_replication_factor();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof SettableBlobMeta)
+      return this.equals((SettableBlobMeta)that);
+    return false;
+  }
+
+  public boolean equals(SettableBlobMeta that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_acl = true && this.is_set_acl();
+    boolean that_present_acl = true && that.is_set_acl();
+    if (this_present_acl || that_present_acl) {
+      if (!(this_present_acl && that_present_acl))
+        return false;
+      if (!this.acl.equals(that.acl))
+        return false;
+    }
+
+    boolean this_present_replication_factor = true && this.is_set_replication_factor();
+    boolean that_present_replication_factor = true && that.is_set_replication_factor();
+    if (this_present_replication_factor || that_present_replication_factor) {
+      if (!(this_present_replication_factor && that_present_replication_factor))
+        return false;
+      if (this.replication_factor != that.replication_factor)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_acl = true && (is_set_acl());
+    list.add(present_acl);
+    if (present_acl)
+      list.add(acl);
+
+    boolean present_replication_factor = true && (is_set_replication_factor());
+    list.add(present_replication_factor);
+    if (present_replication_factor)
+      list.add(replication_factor);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(SettableBlobMeta other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_acl()).compareTo(other.is_set_acl());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_acl()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.acl, other.acl);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_replication_factor()).compareTo(other.is_set_replication_factor());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_replication_factor()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.replication_factor, other.replication_factor);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("SettableBlobMeta(");
+    boolean first = true;
+
+    sb.append("acl:");
+    if (this.acl == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.acl);
+    }
+    first = false;
+    if (is_set_replication_factor()) {
+      if (!first) sb.append(", ");
+      sb.append("replication_factor:");
+      sb.append(this.replication_factor);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_acl()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'acl' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class SettableBlobMetaStandardSchemeFactory implements SchemeFactory {
+    public SettableBlobMetaStandardScheme getScheme() {
+      return new SettableBlobMetaStandardScheme();
+    }
+  }
+
+  private static class SettableBlobMetaStandardScheme extends StandardScheme<SettableBlobMeta> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, SettableBlobMeta struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // ACL
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list488 = iprot.readListBegin();
+                struct.acl = new ArrayList<AccessControl>(_list488.size);
+                AccessControl _elem489;
+                for (int _i490 = 0; _i490 < _list488.size; ++_i490)
+                {
+                  _elem489 = new AccessControl();
+                  _elem489.read(iprot);
+                  struct.acl.add(_elem489);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_acl_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // REPLICATION_FACTOR
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.replication_factor = iprot.readI32();
+              struct.set_replication_factor_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, SettableBlobMeta struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.acl != null) {
+        oprot.writeFieldBegin(ACL_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.acl.size()));
+          for (AccessControl _iter491 : struct.acl)
+          {
+            _iter491.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_replication_factor()) {
+        oprot.writeFieldBegin(REPLICATION_FACTOR_FIELD_DESC);
+        oprot.writeI32(struct.replication_factor);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class SettableBlobMetaTupleSchemeFactory implements SchemeFactory {
+    public SettableBlobMetaTupleScheme getScheme() {
+      return new SettableBlobMetaTupleScheme();
+    }
+  }
+
+  private static class SettableBlobMetaTupleScheme extends TupleScheme<SettableBlobMeta> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, SettableBlobMeta struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.acl.size());
+        for (AccessControl _iter492 : struct.acl)
+        {
+          _iter492.write(oprot);
+        }
+      }
+      BitSet optionals = new BitSet();
+      if (struct.is_set_replication_factor()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_replication_factor()) {
+        oprot.writeI32(struct.replication_factor);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, SettableBlobMeta struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TList _list493 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.acl = new ArrayList<AccessControl>(_list493.size);
+        AccessControl _elem494;
+        for (int _i495 = 0; _i495 < _list493.size; ++_i495)
+        {
+          _elem494 = new AccessControl();
+          _elem494.read(iprot);
+          struct.acl.add(_elem494);
+        }
+      }
+      struct.set_acl_isSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.replication_factor = iprot.readI32();
+        struct.set_replication_factor_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/generated/StormBase.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/StormBase.java b/storm-core/src/jvm/backtype/storm/generated/StormBase.java
index 5f80c59..1bc6ff9 100644
--- a/storm-core/src/jvm/backtype/storm/generated/StormBase.java
+++ b/storm-core/src/jvm/backtype/storm/generated/StormBase.java
@@ -1090,15 +1090,15 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
           case 4: // COMPONENT_EXECUTORS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map588 = iprot.readMapBegin();
-                struct.component_executors = new HashMap<String,Integer>(2*_map588.size);
-                String _key589;
-                int _val590;
-                for (int _i591 = 0; _i591 < _map588.size; ++_i591)
+                org.apache.thrift.protocol.TMap _map604 = iprot.readMapBegin();
+                struct.component_executors = new HashMap<String,Integer>(2*_map604.size);
+                String _key605;
+                int _val606;
+                for (int _i607 = 0; _i607 < _map604.size; ++_i607)
                 {
-                  _key589 = iprot.readString();
-                  _val590 = iprot.readI32();
-                  struct.component_executors.put(_key589, _val590);
+                  _key605 = iprot.readString();
+                  _val606 = iprot.readI32();
+                  struct.component_executors.put(_key605, _val606);
                 }
                 iprot.readMapEnd();
               }
@@ -1143,16 +1143,16 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
           case 9: // COMPONENT_DEBUG
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map592 = iprot.readMapBegin();
-                struct.component_debug = new HashMap<String,DebugOptions>(2*_map592.size);
-                String _key593;
-                DebugOptions _val594;
-                for (int _i595 = 0; _i595 < _map592.size; ++_i595)
+                org.apache.thrift.protocol.TMap _map608 = iprot.readMapBegin();
+                struct.component_debug = new HashMap<String,DebugOptions>(2*_map608.size);
+                String _key609;
+                DebugOptions _val610;
+                for (int _i611 = 0; _i611 < _map608.size; ++_i611)
                 {
-                  _key593 = iprot.readString();
-                  _val594 = new DebugOptions();
-                  _val594.read(iprot);
-                  struct.component_debug.put(_key593, _val594);
+                  _key609 = iprot.readString();
+                  _val610 = new DebugOptions();
+                  _val610.read(iprot);
+                  struct.component_debug.put(_key609, _val610);
                 }
                 iprot.readMapEnd();
               }
@@ -1192,10 +1192,10 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
           oprot.writeFieldBegin(COMPONENT_EXECUTORS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.component_executors.size()));
-            for (Map.Entry<String, Integer> _iter596 : struct.component_executors.entrySet())
+            for (Map.Entry<String, Integer> _iter612 : struct.component_executors.entrySet())
             {
-              oprot.writeString(_iter596.getKey());
-              oprot.writeI32(_iter596.getValue());
+              oprot.writeString(_iter612.getKey());
+              oprot.writeI32(_iter612.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -1233,10 +1233,10 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
           oprot.writeFieldBegin(COMPONENT_DEBUG_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.component_debug.size()));
-            for (Map.Entry<String, DebugOptions> _iter597 : struct.component_debug.entrySet())
+            for (Map.Entry<String, DebugOptions> _iter613 : struct.component_debug.entrySet())
             {
-              oprot.writeString(_iter597.getKey());
-              _iter597.getValue().write(oprot);
+              oprot.writeString(_iter613.getKey());
+              _iter613.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -1286,10 +1286,10 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       if (struct.is_set_component_executors()) {
         {
           oprot.writeI32(struct.component_executors.size());
-          for (Map.Entry<String, Integer> _iter598 : struct.component_executors.entrySet())
+          for (Map.Entry<String, Integer> _iter614 : struct.component_executors.entrySet())
           {
-            oprot.writeString(_iter598.getKey());
-            oprot.writeI32(_iter598.getValue());
+            oprot.writeString(_iter614.getKey());
+            oprot.writeI32(_iter614.getValue());
           }
         }
       }
@@ -1308,10 +1308,10 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       if (struct.is_set_component_debug()) {
         {
           oprot.writeI32(struct.component_debug.size());
-          for (Map.Entry<String, DebugOptions> _iter599 : struct.component_debug.entrySet())
+          for (Map.Entry<String, DebugOptions> _iter615 : struct.component_debug.entrySet())
           {
-            oprot.writeString(_iter599.getKey());
-            _iter599.getValue().write(oprot);
+            oprot.writeString(_iter615.getKey());
+            _iter615.getValue().write(oprot);
           }
         }
       }
@@ -1329,15 +1329,15 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       BitSet incoming = iprot.readBitSet(6);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TMap _map600 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32());
-          struct.component_executors = new HashMap<String,Integer>(2*_map600.size);
-          String _key601;
-          int _val602;
-          for (int _i603 = 0; _i603 < _map600.size; ++_i603)
+          org.apache.thrift.protocol.TMap _map616 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32());
+          struct.component_executors = new HashMap<String,Integer>(2*_map616.size);
+          String _key617;
+          int _val618;
+          for (int _i619 = 0; _i619 < _map616.size; ++_i619)
           {
-            _key601 = iprot.readString();
-            _val602 = iprot.readI32();
-            struct.component_executors.put(_key601, _val602);
+            _key617 = iprot.readString();
+            _val618 = iprot.readI32();
+            struct.component_executors.put(_key617, _val618);
           }
         }
         struct.set_component_executors_isSet(true);
@@ -1361,16 +1361,16 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       }
       if (incoming.get(5)) {
         {
-          org.apache.thrift.protocol.TMap _map604 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.component_debug = new HashMap<String,DebugOptions>(2*_map604.size);
-          String _key605;
-          DebugOptions _val606;
-          for (int _i607 = 0; _i607 < _map604.size; ++_i607)
+          org.apache.thrift.protocol.TMap _map620 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.component_debug = new HashMap<String,DebugOptions>(2*_map620.size);
+          String _key621;
+          DebugOptions _val622;
+          for (int _i623 = 0; _i623 < _map620.size; ++_i623)
           {
-            _key605 = iprot.readString();
-            _val606 = new DebugOptions();
-            _val606.read(iprot);
-            struct.component_debug.put(_key605, _val606);
+            _key621 = iprot.readString();
+            _val622 = new DebugOptions();
+            _val622.read(iprot);
+            struct.component_debug.put(_key621, _val622);
           }
         }
         struct.set_component_debug_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java b/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java
index 2ce5eb9..4d77c08 100644
--- a/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java
@@ -1085,13 +1085,13 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
           case 4: // USED_PORTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list488 = iprot.readListBegin();
-                struct.used_ports = new ArrayList<Long>(_list488.size);
-                long _elem489;
-                for (int _i490 = 0; _i490 < _list488.size; ++_i490)
+                org.apache.thrift.protocol.TList _list504 = iprot.readListBegin();
+                struct.used_ports = new ArrayList<Long>(_list504.size);
+                long _elem505;
+                for (int _i506 = 0; _i506 < _list504.size; ++_i506)
                 {
-                  _elem489 = iprot.readI64();
-                  struct.used_ports.add(_elem489);
+                  _elem505 = iprot.readI64();
+                  struct.used_ports.add(_elem505);
                 }
                 iprot.readListEnd();
               }
@@ -1103,13 +1103,13 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
           case 5: // META
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list491 = iprot.readListBegin();
-                struct.meta = new ArrayList<Long>(_list491.size);
-                long _elem492;
-                for (int _i493 = 0; _i493 < _list491.size; ++_i493)
+                org.apache.thrift.protocol.TList _list507 = iprot.readListBegin();
+                struct.meta = new ArrayList<Long>(_list507.size);
+                long _elem508;
+                for (int _i509 = 0; _i509 < _list507.size; ++_i509)
                 {
-                  _elem492 = iprot.readI64();
-                  struct.meta.add(_elem492);
+                  _elem508 = iprot.readI64();
+                  struct.meta.add(_elem508);
                 }
                 iprot.readListEnd();
               }
@@ -1121,15 +1121,15 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
           case 6: // SCHEDULER_META
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map494 = iprot.readMapBegin();
-                struct.scheduler_meta = new HashMap<String,String>(2*_map494.size);
-                String _key495;
-                String _val496;
-                for (int _i497 = 0; _i497 < _map494.size; ++_i497)
+                org.apache.thrift.protocol.TMap _map510 = iprot.readMapBegin();
+                struct.scheduler_meta = new HashMap<String,String>(2*_map510.size);
+                String _key511;
+                String _val512;
+                for (int _i513 = 0; _i513 < _map510.size; ++_i513)
                 {
-                  _key495 = iprot.readString();
-                  _val496 = iprot.readString();
-                  struct.scheduler_meta.put(_key495, _val496);
+                  _key511 = iprot.readString();
+                  _val512 = iprot.readString();
+                  struct.scheduler_meta.put(_key511, _val512);
                 }
                 iprot.readMapEnd();
               }
@@ -1157,15 +1157,15 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
           case 9: // RESOURCES_MAP
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map498 = iprot.readMapBegin();
-                struct.resources_map = new HashMap<String,Double>(2*_map498.size);
-                String _key499;
-                double _val500;
-                for (int _i501 = 0; _i501 < _map498.size; ++_i501)
+                org.apache.thrift.protocol.TMap _map514 = iprot.readMapBegin();
+                struct.resources_map = new HashMap<String,Double>(2*_map514.size);
+                String _key515;
+                double _val516;
+                for (int _i517 = 0; _i517 < _map514.size; ++_i517)
                 {
-                  _key499 = iprot.readString();
-                  _val500 = iprot.readDouble();
-                  struct.resources_map.put(_key499, _val500);
+                  _key515 = iprot.readString();
+                  _val516 = iprot.readDouble();
+                  struct.resources_map.put(_key515, _val516);
                 }
                 iprot.readMapEnd();
               }
@@ -1207,9 +1207,9 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
           oprot.writeFieldBegin(USED_PORTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.used_ports.size()));
-            for (long _iter502 : struct.used_ports)
+            for (long _iter518 : struct.used_ports)
             {
-              oprot.writeI64(_iter502);
+              oprot.writeI64(_iter518);
             }
             oprot.writeListEnd();
           }
@@ -1221,9 +1221,9 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
           oprot.writeFieldBegin(META_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.meta.size()));
-            for (long _iter503 : struct.meta)
+            for (long _iter519 : struct.meta)
             {
-              oprot.writeI64(_iter503);
+              oprot.writeI64(_iter519);
             }
             oprot.writeListEnd();
           }
@@ -1235,10 +1235,10 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
           oprot.writeFieldBegin(SCHEDULER_META_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.scheduler_meta.size()));
-            for (Map.Entry<String, String> _iter504 : struct.scheduler_meta.entrySet())
+            for (Map.Entry<String, String> _iter520 : struct.scheduler_meta.entrySet())
             {
-              oprot.writeString(_iter504.getKey());
-              oprot.writeString(_iter504.getValue());
+              oprot.writeString(_iter520.getKey());
+              oprot.writeString(_iter520.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -1262,10 +1262,10 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
           oprot.writeFieldBegin(RESOURCES_MAP_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, struct.resources_map.size()));
-            for (Map.Entry<String, Double> _iter505 : struct.resources_map.entrySet())
+            for (Map.Entry<String, Double> _iter521 : struct.resources_map.entrySet())
             {
-              oprot.writeString(_iter505.getKey());
-              oprot.writeDouble(_iter505.getValue());
+              oprot.writeString(_iter521.getKey());
+              oprot.writeDouble(_iter521.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -1320,28 +1320,28 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
       if (struct.is_set_used_ports()) {
         {
           oprot.writeI32(struct.used_ports.size());
-          for (long _iter506 : struct.used_ports)
+          for (long _iter522 : struct.used_ports)
           {
-            oprot.writeI64(_iter506);
+            oprot.writeI64(_iter522);
           }
         }
       }
       if (struct.is_set_meta()) {
         {
           oprot.writeI32(struct.meta.size());
-          for (long _iter507 : struct.meta)
+          for (long _iter523 : struct.meta)
           {
-            oprot.writeI64(_iter507);
+            oprot.writeI64(_iter523);
           }
         }
       }
       if (struct.is_set_scheduler_meta()) {
         {
           oprot.writeI32(struct.scheduler_meta.size());
-          for (Map.Entry<String, String> _iter508 : struct.scheduler_meta.entrySet())
+          for (Map.Entry<String, String> _iter524 : struct.scheduler_meta.entrySet())
           {
-            oprot.writeString(_iter508.getKey());
-            oprot.writeString(_iter508.getValue());
+            oprot.writeString(_iter524.getKey());
+            oprot.writeString(_iter524.getValue());
           }
         }
       }
@@ -1354,10 +1354,10 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
       if (struct.is_set_resources_map()) {
         {
           oprot.writeI32(struct.resources_map.size());
-          for (Map.Entry<String, Double> _iter509 : struct.resources_map.entrySet())
+          for (Map.Entry<String, Double> _iter525 : struct.resources_map.entrySet())
           {
-            oprot.writeString(_iter509.getKey());
-            oprot.writeDouble(_iter509.getValue());
+            oprot.writeString(_iter525.getKey());
+            oprot.writeDouble(_iter525.getValue());
           }
         }
       }
@@ -1377,41 +1377,41 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list510 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-          struct.used_ports = new ArrayList<Long>(_list510.size);
-          long _elem511;
-          for (int _i512 = 0; _i512 < _list510.size; ++_i512)
+          org.apache.thrift.protocol.TList _list526 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.used_ports = new ArrayList<Long>(_list526.size);
+          long _elem527;
+          for (int _i528 = 0; _i528 < _list526.size; ++_i528)
           {
-            _elem511 = iprot.readI64();
-            struct.used_ports.add(_elem511);
+            _elem527 = iprot.readI64();
+            struct.used_ports.add(_elem527);
           }
         }
         struct.set_used_ports_isSet(true);
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list513 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-          struct.meta = new ArrayList<Long>(_list513.size);
-          long _elem514;
-          for (int _i515 = 0; _i515 < _list513.size; ++_i515)
+          org.apache.thrift.protocol.TList _list529 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.meta = new ArrayList<Long>(_list529.size);
+          long _elem530;
+          for (int _i531 = 0; _i531 < _list529.size; ++_i531)
           {
-            _elem514 = iprot.readI64();
-            struct.meta.add(_elem514);
+            _elem530 = iprot.readI64();
+            struct.meta.add(_elem530);
           }
         }
         struct.set_meta_isSet(true);
       }
       if (incoming.get(3)) {
         {
-          org.apache.thrift.protocol.TMap _map516 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.scheduler_meta = new HashMap<String,String>(2*_map516.size);
-          String _key517;
-          String _val518;
-          for (int _i519 = 0; _i519 < _map516.size; ++_i519)
+          org.apache.thrift.protocol.TMap _map532 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.scheduler_meta = new HashMap<String,String>(2*_map532.size);
+          String _key533;
+          String _val534;
+          for (int _i535 = 0; _i535 < _map532.size; ++_i535)
           {
-            _key517 = iprot.readString();
-            _val518 = iprot.readString();
-            struct.scheduler_meta.put(_key517, _val518);
+            _key533 = iprot.readString();
+            _val534 = iprot.readString();
+            struct.scheduler_meta.put(_key533, _val534);
           }
         }
         struct.set_scheduler_meta_isSet(true);
@@ -1426,15 +1426,15 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
       }
       if (incoming.get(6)) {
         {
-          org.apache.thrift.protocol.TMap _map520 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
-          struct.resources_map = new HashMap<String,Double>(2*_map520.size);
-          String _key521;
-          double _val522;
-          for (int _i523 = 0; _i523 < _map520.size; ++_i523)
+          org.apache.thrift.protocol.TMap _map536 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
+          struct.resources_map = new HashMap<String,Double>(2*_map536.size);
+          String _key537;
+          double _val538;
+          for (int _i539 = 0; _i539 < _map536.size; ++_i539)
           {
-            _key521 = iprot.readString();
-            _val522 = iprot.readDouble();
-            struct.resources_map.put(_key521, _val522);
+            _key537 = iprot.readString();
+            _val538 = iprot.readDouble();
+            struct.resources_map.put(_key537, _val538);
           }
         }
         struct.set_resources_map_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/generated/TopologyHistoryInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyHistoryInfo.java b/storm-core/src/jvm/backtype/storm/generated/TopologyHistoryInfo.java
index cced456..cc8ef3c 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologyHistoryInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyHistoryInfo.java
@@ -364,13 +364,13 @@ public class TopologyHistoryInfo implements org.apache.thrift.TBase<TopologyHist
           case 1: // TOPO_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list698 = iprot.readListBegin();
-                struct.topo_ids = new ArrayList<String>(_list698.size);
-                String _elem699;
-                for (int _i700 = 0; _i700 < _list698.size; ++_i700)
+                org.apache.thrift.protocol.TList _list714 = iprot.readListBegin();
+                struct.topo_ids = new ArrayList<String>(_list714.size);
+                String _elem715;
+                for (int _i716 = 0; _i716 < _list714.size; ++_i716)
                 {
-                  _elem699 = iprot.readString();
-                  struct.topo_ids.add(_elem699);
+                  _elem715 = iprot.readString();
+                  struct.topo_ids.add(_elem715);
                 }
                 iprot.readListEnd();
               }
@@ -396,9 +396,9 @@ public class TopologyHistoryInfo implements org.apache.thrift.TBase<TopologyHist
         oprot.writeFieldBegin(TOPO_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.topo_ids.size()));
-          for (String _iter701 : struct.topo_ids)
+          for (String _iter717 : struct.topo_ids)
           {
-            oprot.writeString(_iter701);
+            oprot.writeString(_iter717);
           }
           oprot.writeListEnd();
         }
@@ -429,9 +429,9 @@ public class TopologyHistoryInfo implements org.apache.thrift.TBase<TopologyHist
       if (struct.is_set_topo_ids()) {
         {
           oprot.writeI32(struct.topo_ids.size());
-          for (String _iter702 : struct.topo_ids)
+          for (String _iter718 : struct.topo_ids)
           {
-            oprot.writeString(_iter702);
+            oprot.writeString(_iter718);
           }
         }
       }
@@ -443,13 +443,13 @@ public class TopologyHistoryInfo implements org.apache.thrift.TBase<TopologyHist
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list703 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.topo_ids = new ArrayList<String>(_list703.size);
-          String _elem704;
-          for (int _i705 = 0; _i705 < _list703.size; ++_i705)
+          org.apache.thrift.protocol.TList _list719 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.topo_ids = new ArrayList<String>(_list719.size);
+          String _elem720;
+          for (int _i721 = 0; _i721 < _list719.size; ++_i721)
           {
-            _elem704 = iprot.readString();
-            struct.topo_ids.add(_elem704);
+            _elem720 = iprot.readString();
+            struct.topo_ids.add(_elem720);
           }
         }
         struct.set_topo_ids_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/localizer/LocalResource.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/localizer/LocalResource.java b/storm-core/src/jvm/backtype/storm/localizer/LocalResource.java
new file mode 100644
index 0000000..beebbd2
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/localizer/LocalResource.java
@@ -0,0 +1,44 @@
+/**
+ * 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.localizer;
+
+/**
+ * Local Resource requested by the topology
+ */
+public class LocalResource {
+  private String _blobKey;
+  private boolean _uncompress;
+
+  public LocalResource(String keyname, boolean uncompress) {
+    _blobKey = keyname;
+    _uncompress = uncompress;
+  }
+
+  public String getBlobName() {
+    return _blobKey;
+  }
+
+  public boolean shouldUncompress() {
+    return _uncompress;
+  }
+
+  @Override
+  public String toString() {
+    return "Key: " + _blobKey + " uncompress: " + _uncompress;
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/localizer/LocalizedResource.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/localizer/LocalizedResource.java b/storm-core/src/jvm/backtype/storm/localizer/LocalizedResource.java
new file mode 100644
index 0000000..3550d36
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/localizer/LocalizedResource.java
@@ -0,0 +1,130 @@
+/**
+ * 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.localizer;
+
+import backtype.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * Represents a resource that is localized on the supervisor.
+ * A localized resource has a .current symlink to the current version file which is named
+ * filename.{current version}. There is also a filename.version which contains the latest version.
+ */
+public class LocalizedResource {
+  public static final Logger LOG = LoggerFactory.getLogger(LocalizedResource.class);
+
+  // filesystem path to the resource
+  private final String _localPath;
+  private final String _versionFilePath;
+  private final String _symlinkPath;
+  private final String _key;
+  private final boolean _uncompressed;
+  // _size of the resource
+  private long _size = -1;
+  // queue of topologies referencing resource
+  private final Set<String> _ref;
+  // last access time of the resource -> increment when topology finishes using it
+  private final AtomicLong _lastAccessTime = new AtomicLong(currentTime());
+
+  public LocalizedResource(String key, String fileLoc, boolean uncompressed) {
+    _ref = new HashSet<String>();
+    _localPath = fileLoc;
+    _versionFilePath = Utils.constructVersionFileName(fileLoc);
+    _symlinkPath = Utils.constructBlobCurrentSymlinkName(fileLoc);
+    _uncompressed = uncompressed;
+    _key = key;
+    // we trust that the file exists
+    _size = Utils.getDU(new File(getFilePathWithVersion()));
+    LOG.debug("size of {} is: {}", fileLoc, _size);
+  }
+
+  // create local resource and add reference
+  public LocalizedResource(String key, String fileLoc, boolean uncompressed, String topo) {
+    this(key, fileLoc, uncompressed);
+    _ref.add(topo);
+  }
+
+  public boolean isUncompressed() {
+    return _uncompressed;
+  }
+
+  public String getKey() {
+    return _key;
+  }
+
+  public String getCurrentSymlinkPath() {
+    return _symlinkPath;
+  }
+
+  public String getVersionFilePath() {
+    return _versionFilePath;
+  }
+
+  public String getFilePathWithVersion() {
+    long version = Utils.localVersionOfBlob(_localPath);
+    return Utils.constructBlobWithVersionFileName(_localPath, version);
+  }
+
+  public String getFilePath() {
+    return _localPath;
+  }
+
+  public void addReference(String topo) {
+    _ref.add(topo);
+  }
+
+  public void removeReference(String topo) {
+    if (!_ref.remove(topo)) {
+      LOG.warn("Tried to remove a reference to a topology that doesn't use this resource");
+    }
+    setTimestamp();
+  }
+
+  // The last access time is only valid if the resource doesn't have any references.
+  public long getLastAccessTime() {
+    return _lastAccessTime.get();
+  }
+
+  // for testing
+  protected void setSize(long size) {
+    _size = size;
+  }
+
+  public long getSize() {
+    return _size;
+  }
+
+  private void setTimestamp() {
+    _lastAccessTime.set(currentTime());
+  }
+
+  public int getRefCount() {
+    return _ref.size();
+  }
+
+  private long currentTime() {
+    return System.nanoTime();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/localizer/LocalizedResourceRetentionSet.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/localizer/LocalizedResourceRetentionSet.java b/storm-core/src/jvm/backtype/storm/localizer/LocalizedResourceRetentionSet.java
new file mode 100644
index 0000000..29cc5a2
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/localizer/LocalizedResourceRetentionSet.java
@@ -0,0 +1,140 @@
+/**
+ * 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.localizer;
+
+import org.apache.commons.io.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+/**
+ * A set of resources that we can look at to see which ones we retain and which ones should be
+ * removed.
+ */
+public class LocalizedResourceRetentionSet {
+  public static final Logger LOG = LoggerFactory.getLogger(LocalizedResourceRetentionSet.class);
+  private long _delSize;
+  private long _currentSize;
+  // targetSize in Bytes
+  private long _targetSize;
+  private final SortedMap<LocalizedResource, LocalizedResourceSet> _noReferences;
+
+  LocalizedResourceRetentionSet(long targetSize) {
+    this(targetSize, new LRUComparator());
+  }
+
+  LocalizedResourceRetentionSet(long targetSize, Comparator<? super LocalizedResource> cmp) {
+    this(targetSize, new TreeMap<LocalizedResource, LocalizedResourceSet>(cmp));
+  }
+
+  LocalizedResourceRetentionSet(long targetSize,
+                                SortedMap<LocalizedResource, LocalizedResourceSet> retain) {
+    this._noReferences = retain;
+    this._targetSize = targetSize;
+  }
+
+  // for testing
+  protected int getSizeWithNoReferences() {
+    return _noReferences.size();
+  }
+
+  protected void addResourcesForSet(Iterator<LocalizedResource> setIter, LocalizedResourceSet set) {
+    for (Iterator<LocalizedResource> iter = setIter; setIter.hasNext(); ) {
+      LocalizedResource lrsrc = iter.next();
+      _currentSize += lrsrc.getSize();
+      if (lrsrc.getRefCount() > 0) {
+        // always retain resources in use
+        continue;
+      }
+      LOG.debug("adding {} to be checked for cleaning", lrsrc.getKey());
+      _noReferences.put(lrsrc, set);
+    }
+  }
+
+  public void addResources(LocalizedResourceSet set) {
+    addResourcesForSet(set.getLocalFilesIterator(), set);
+    addResourcesForSet(set.getLocalArchivesIterator(), set);
+  }
+
+  public void cleanup() {
+    LOG.debug("cleanup target size: {} current size is: {}", _targetSize, _currentSize);
+    for (Iterator<Map.Entry<LocalizedResource, LocalizedResourceSet>> i =
+             _noReferences.entrySet().iterator();
+         _currentSize - _delSize > _targetSize && i.hasNext();) {
+      Map.Entry<LocalizedResource, LocalizedResourceSet> rsrc = i.next();
+      LocalizedResource resource = rsrc.getKey();
+      LocalizedResourceSet set = rsrc.getValue();
+      if (resource != null && set.remove(resource)) {
+        if (deleteResource(resource)) {
+          _delSize += resource.getSize();
+          LOG.info("deleting: " + resource.getFilePath() + " size of: " + resource.getSize());
+          i.remove();
+        } else {
+          // since it failed to delete add it back so it gets retried
+          set.addResource(resource.getKey(), resource, resource.isUncompressed());
+        }
+      }
+    }
+  }
+
+  protected boolean deleteResource(LocalizedResource resource){
+    try {
+      String fileWithVersion = resource.getFilePathWithVersion();
+      String currentSymlinkName = resource.getCurrentSymlinkPath();
+      String versionFile = resource.getVersionFilePath();
+      File deletePath = new File(fileWithVersion);
+      if (resource.isUncompressed()) {
+        // this doesn't follow symlinks, which is what we want
+        FileUtils.deleteDirectory(deletePath);
+      } else {
+        Files.delete(deletePath.toPath());
+      }
+      Files.delete(new File(currentSymlinkName).toPath());
+      Files.delete(new File(versionFile).toPath());
+      return true;
+    } catch (IOException e) {
+      LOG.warn("Could not delete: {}", resource.getFilePath());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("Cache: ").append(_currentSize).append(", ");
+    sb.append("Deleted: ").append(_delSize);
+    return sb.toString();
+  }
+
+  static class LRUComparator implements Comparator<LocalizedResource> {
+    public int compare(LocalizedResource r1, LocalizedResource r2) {
+      long ret = r1.getLastAccessTime() - r2.getLastAccessTime();
+      if (0 == ret) {
+        return System.identityHashCode(r1) - System.identityHashCode(r2);
+      }
+      return ret > 0 ? 1 : -1;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7029aee5/storm-core/src/jvm/backtype/storm/localizer/LocalizedResourceSet.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/localizer/LocalizedResourceSet.java b/storm-core/src/jvm/backtype/storm/localizer/LocalizedResourceSet.java
new file mode 100644
index 0000000..fa5e54c
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/localizer/LocalizedResourceSet.java
@@ -0,0 +1,101 @@
+/**
+ * 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.localizer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Iterator;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * Set of localized resources for a specific user.
+ */
+public class LocalizedResourceSet {
+
+  public static final Logger LOG = LoggerFactory.getLogger(LocalizedResourceSet.class);
+  // Key to LocalizedResource mapping for files
+  private final ConcurrentMap<String, LocalizedResource> _localrsrcFiles;
+  // Key to LocalizedResource mapping for files to be uncompressed
+  private final ConcurrentMap<String, LocalizedResource> _localrsrcArchives;
+  private String _user;
+
+  LocalizedResourceSet(String user) {
+    this._localrsrcFiles = new ConcurrentHashMap<String, LocalizedResource>();
+    this._localrsrcArchives = new ConcurrentHashMap<String, LocalizedResource>();
+    _user = user;
+  }
+
+  public String getUser() {
+    return _user;
+  }
+
+  public int getSize() {
+    return _localrsrcFiles.size() + _localrsrcArchives.size();
+  }
+
+  public LocalizedResource get(String name, boolean uncompress) {
+    if (uncompress) {
+      return _localrsrcArchives.get(name);
+    }
+    return _localrsrcFiles.get(name);
+  }
+
+  public void updateResource(String resourceName, LocalizedResource updatedResource,
+                            boolean uncompress) {
+    if (uncompress) {
+      _localrsrcArchives.putIfAbsent(resourceName, updatedResource);
+    } else {
+      _localrsrcFiles.putIfAbsent(resourceName, updatedResource);
+    }
+  }
+
+  public void addResource(String resourceName, LocalizedResource newResource, boolean uncompress) {
+    if (uncompress) {
+      _localrsrcArchives.put(resourceName, newResource);
+    } else {
+      _localrsrcFiles.put(resourceName, newResource);
+    }
+  }
+
+  public boolean exists(String resourceName, boolean uncompress) {
+    if (uncompress) {
+      return (_localrsrcArchives.get(resourceName) != null);
+    }
+    return (_localrsrcFiles.get(resourceName) != null);
+  }
+
+  public boolean remove(LocalizedResource resource) {
+    LocalizedResource lrsrc = null;
+    if (resource.isUncompressed()) {
+      lrsrc = _localrsrcArchives.remove(resource.getKey());
+    } else {
+      lrsrc = _localrsrcFiles.remove(resource.getKey());
+    }
+    return (lrsrc != null);
+  }
+
+  public Iterator<LocalizedResource> getLocalFilesIterator() {
+    return _localrsrcFiles.values().iterator();
+  }
+
+  public Iterator<LocalizedResource> getLocalArchivesIterator() {
+    return _localrsrcArchives.values().iterator();
+  }
+}