[
https://issues.apache.org/jira/browse/STORM-876?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15031079#comment-15031079
]
ASF GitHub Bot commented on STORM-876:
--------------------------------------
Github user redsanket commented on a diff in the pull request:
https://github.com/apache/storm/pull/845#discussion_r46094594
--- Diff: storm-core/src/jvm/backtype/storm/blobstore/BlobStoreUtils.java
---
@@ -0,0 +1,256 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.blobstore;
+
+import backtype.storm.Config;
+import backtype.storm.generated.AuthorizationException;
+import backtype.storm.generated.KeyAlreadyExistsException;
+import backtype.storm.generated.KeyNotFoundException;
+import backtype.storm.generated.ReadableBlobMeta;
+import backtype.storm.nimbus.NimbusInfo;
+import backtype.storm.security.auth.NimbusPrincipal;
+import backtype.storm.utils.NimbusClient;
+import backtype.storm.utils.Utils;
+import backtype.storm.utils.ZookeeperAuthInfo;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.security.auth.Subject;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class BlobStoreUtils {
+ private static final String BLOBSTORE_SUBTREE="/blobstore";
+ private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
+
+ public static CuratorFramework createZKClient(Map conf) {
+ List<String> zkServers = (List<String>)
conf.get(Config.STORM_ZOOKEEPER_SERVERS);
+ Object port = conf.get(Config.STORM_ZOOKEEPER_PORT);
+ ZookeeperAuthInfo zkAuthInfo = new ZookeeperAuthInfo(conf);
+ CuratorFramework zkClient = Utils.newCurator(conf, zkServers,
port, (String) conf.get(Config.STORM_ZOOKEEPER_ROOT), zkAuthInfo);
+ zkClient.start();
+ return zkClient;
+ }
+
+ public static Subject getNimbusSubject() {
+ Subject subject = new Subject();
+ subject.getPrincipals().add(new NimbusPrincipal());
+ return subject;
+ }
+
+ // Normalize state
+ public static BlobKeySequenceInfo
normalizeNimbusHostPortSequenceNumberInfo(String nimbusSeqNumberInfo) {
+ BlobKeySequenceInfo keySequenceInfo = new BlobKeySequenceInfo();
+ int lastIndex = nimbusSeqNumberInfo.lastIndexOf("-");
+ keySequenceInfo.setNimbusHostPort(nimbusSeqNumberInfo.substring(0,
lastIndex));
+
keySequenceInfo.setSequenceNumber(nimbusSeqNumberInfo.substring(lastIndex + 1));
+ return keySequenceInfo;
+ }
+
+ // Check for latest sequence number of a key inside zookeeper and
return nimbodes containing the latest sequence number
+ public static Set<NimbusInfo>
getNimbodesWithLatestSequenceNumberOfBlob(CuratorFramework zkClient, String
key) throws Exception {
+ List<String> stateInfoList =
zkClient.getChildren().forPath("/blobstore/" + key);
+ Set<NimbusInfo> nimbusInfoSet = new HashSet<NimbusInfo>();
+ int latestSeqNumber = getLatestSequenceNumber(stateInfoList);
+ LOG.debug("getNimbodesWithLatestSequenceNumberOfBlob stateInfo {}
version {}", stateInfoList, latestSeqNumber);
+ // Get the nimbodes with the latest version
+ for(String state : stateInfoList) {
+ BlobKeySequenceInfo sequenceInfo =
normalizeNimbusHostPortSequenceNumberInfo(state);
+ if (latestSeqNumber ==
Integer.parseInt(sequenceInfo.getSequenceNumber())) {
+
nimbusInfoSet.add(NimbusInfo.parse(sequenceInfo.getNimbusHostPort()));
+ }
+ }
+ LOG.debug("nimbusInfoList {}", nimbusInfoSet);
+ return nimbusInfoSet;
+ }
+
+ // Get sequence number details from latest sequence number of the blob
+ public static int getLatestSequenceNumber(List<String> stateInfoList) {
+ int seqNumber = 0;
+ // Get latest sequence number of the blob present in the zookeeper
--> possible to refactor this piece of code
+ for (String state : stateInfoList) {
+ BlobKeySequenceInfo sequenceInfo =
normalizeNimbusHostPortSequenceNumberInfo(state);
+ if (seqNumber <
Integer.parseInt(sequenceInfo.getSequenceNumber())) {
+ seqNumber =
Integer.parseInt(sequenceInfo.getSequenceNumber());
+ LOG.debug("Sequence Info {}", seqNumber);
+ }
+ }
+ LOG.debug("Latest Sequence Number {}", seqNumber);
+ return seqNumber;
+ }
+
+ // Download missing blobs from potential nimbodes
+ public static boolean downloadMissingBlob(Map conf, BlobStore
blobStore, String key, Set<NimbusInfo> nimbusInfos)
+ throws TTransportException {
+ NimbusClient client = null;
+ ReadableBlobMeta rbm = null;
+ ClientBlobStore remoteBlobStore = null;
+ InputStreamWithMeta in = null;
+ Boolean isSuccess = false;
+ LOG.debug("Download blob NimbusInfos {}", nimbusInfos);
+ for (NimbusInfo nimbusInfo : nimbusInfos) {
+ if(isSuccess) {
+ break;
+ }
+ try {
+ client = new NimbusClient(conf, nimbusInfo.getHost(),
nimbusInfo.getPort(), null);
+ rbm = client.getClient().getBlobMeta(key);
+ remoteBlobStore = new NimbusBlobStore();
+ remoteBlobStore.setClient(conf, client);
+ in = remoteBlobStore.getBlob(key);
+ blobStore.createBlob(key, in, rbm.get_settable(),
getNimbusSubject());
+ // if key already exists while creating the blob else
update it
+ Iterator<String> keyIterator = blobStore.listKeys();
+ while (keyIterator.hasNext()) {
+ if (keyIterator.next().equals(key)) {
+ LOG.debug("Success creating key, {}", key);
+ isSuccess = true;
+ break;
+ }
+ }
+ } catch (IOException | AuthorizationException exception) {
+ throw new RuntimeException(exception);
+ } catch (KeyAlreadyExistsException kae) {
+ LOG.info("KeyAlreadyExistsException Key: {} {}", key, kae);
+ } catch (KeyNotFoundException knf) {
+ // Catching and logging KeyNotFoundException because, if
+ // there is a subsequent update and delete, the non-leader
+ // nimbodes might throw an exception.
+ LOG.info("KeyNotFoundException Key: {} {}", key, knf);
+ } catch (Exception exp) {
+ // Logging an exception while client is connecting
+ LOG.error("Exception {}", exp);
+ }
+ }
+
+ if (!isSuccess) {
+ LOG.error("Could not download blob with key" + key);
+ return false;
+ }
+ return isSuccess;
+ }
+
+ // Download updated blobs from potential nimbodes
+ public static boolean downloadUpdatedBlob(Map conf, BlobStore
blobStore, String key, Set<NimbusInfo> nimbusInfos)
+ throws TTransportException {
+ NimbusClient client = null;
+ ClientBlobStore remoteBlobStore = null;
+ InputStreamWithMeta in = null;
+ AtomicOutputStream out = null;
+ Boolean isSuccess = false;
+ LOG.debug("Download blob NimbusInfos {}", nimbusInfos);
+ for (NimbusInfo nimbusInfo : nimbusInfos) {
+ if (isSuccess) {
+ break;
+ }
+ try {
+ client = new NimbusClient(conf, nimbusInfo.getHost(),
nimbusInfo.getPort(), null);
+ remoteBlobStore = new NimbusBlobStore();
+ remoteBlobStore.setClient(conf, client);
+ in = remoteBlobStore.getBlob(key);
+ out = blobStore.updateBlob(key, getNimbusSubject());
+ byte[] buffer = new byte[2048];
+ int len = 0;
+ while ((len = in.read(buffer)) > 0) {
+ out.write(buffer, 0, len);
+ }
+ if (out != null) {
+ out.close();
+ }
+ isSuccess = true;
+ } catch (IOException | AuthorizationException exception) {
+ throw new RuntimeException(exception);
+ } catch (KeyNotFoundException knf) {
+ // Catching and logging KeyNotFoundException because, if
+ // there is a subsequent update and delete, the non-leader
+ // nimbodes might throw an exception.
+ LOG.info("KeyNotFoundException {}", knf);
+ } catch (Exception exp) {
+ // Logging an exception while client is connecting
+ LOG.error("Exception {}", exp);
+ }
+ }
+
+ if (!isSuccess) {
+ LOG.error("Could not update the blob with key" + key);
+ return false;
+ }
+ return isSuccess;
+ }
+
+ // Get the list of keys from blobstore
+ public static List<String> getKeyListFromBlobStore(BlobStore
blobStore) throws Exception {
+ Iterator<String> keys = blobStore.listKeys();
+ List<String> keyList = new ArrayList<String>();
+ while (keys != null && keys.hasNext()) {
--- End diff --
I am right, as this is a short circuit operator it will not execute if keys
!= null. I do not think it will make any difference if I move this out. Might
suggest if I am wrong
> Dist Cache: Basic Functionality
> -------------------------------
>
> Key: STORM-876
> URL: https://issues.apache.org/jira/browse/STORM-876
> Project: Apache Storm
> Issue Type: Improvement
> Components: storm-core
> Reporter: Robert Joseph Evans
> Assignee: Robert Joseph Evans
> Attachments: DISTCACHE.md, DistributedCacheDesignDocument.pdf
>
>
> Basic functionality for the Dist Cache feature.
> As part of this a new API should be added to support uploading and
> downloading dist cache items. storm-core.ser, storm-conf.ser and storm.jar
> should be written into the blob store instead of residing locally. We need a
> default implementation of the blob store that does essentially what nimbus
> currently does and does not need anything extra. But having an HDFS backend
> too would be great for scalability and HA.
> The supervisor should provide a way to download and manage these blobs and
> provide a working directory for the worker process with symlinks to the
> blobs. It should also allow the blobs to be updated and switch the symlink
> atomically to point to the new blob once it is downloaded.
> All of this is already done by code internal to Yahoo! we are in the process
> of getting it ready to push back to open source shortly.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)