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();
+ }
+}