[
https://issues.apache.org/jira/browse/STORM-1273?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15130631#comment-15130631
]
ASF GitHub Bot commented on STORM-1273:
---------------------------------------
Github user abhishekagarwal87 commented on a diff in the pull request:
https://github.com/apache/storm/pull/1071#discussion_r51744446
--- Diff:
storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java ---
@@ -0,0 +1,683 @@
+/**
+ * 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.cluster;
+
+import clojure.lang.APersistentMap;
+import clojure.lang.IFn;
+import clojure.lang.PersistentArrayMap;
+import clojure.lang.RT;
+import org.apache.commons.lang.StringUtils;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.state.*;
+import org.apache.curator.framework.state.ConnectionState;
+import org.apache.storm.callback.Callback;
+import org.apache.storm.generated.*;
+import org.apache.storm.nimbus.NimbusInfo;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.zookeeper.Zookeeper;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.UnsupportedEncodingException;
+import java.security.NoSuchAlgorithmException;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+
+public class StormZkClusterState implements StormClusterState {
+
+ private static Logger LOG =
LoggerFactory.getLogger(StormZkClusterState.class);
+
+ private ClusterState clusterState;
+
+ private ConcurrentHashMap<String, IFn> assignmentInfoCallback;
+ private ConcurrentHashMap<String, IFn>
assignmentInfoWithVersionCallback;
+ private ConcurrentHashMap<String, IFn> assignmentVersionCallback;
+ private AtomicReference<IFn> supervisorsCallback;
+ // we want to reigister a topo directory getChildren callback for all
workers of this dir
+ private ConcurrentHashMap<String, IFn> backPressureCallback;
+ private AtomicReference<IFn> assignmentsCallback;
+ private ConcurrentHashMap<String, IFn> stormBaseCallback;
+ private AtomicReference<IFn> blobstoreCallback;
+ private ConcurrentHashMap<String, IFn> credentialsCallback;
+ private ConcurrentHashMap<String, IFn> logConfigCallback;
+
+ private List<ACL> acls;
+ private String stateId;
+ private boolean solo;
+
+ public StormZkClusterState(Object clusterState, List<ACL> acls,
ClusterStateContext context) throws Exception {
+
+ if (clusterState instanceof ClusterState) {
+ solo = false;
+ this.clusterState = (ClusterState) clusterState;
+ } else {
+
+ solo = true;
+ this.clusterState = new DistributedClusterState((Map)
clusterState, (Map) clusterState, acls, context);
+ }
+
+ assignmentInfoCallback = new ConcurrentHashMap<>();
+ assignmentInfoWithVersionCallback = new ConcurrentHashMap<>();
+ assignmentVersionCallback = new ConcurrentHashMap<>();
+ supervisorsCallback = new AtomicReference<>();
+ backPressureCallback = new ConcurrentHashMap<>();
+ assignmentsCallback = new AtomicReference<>();
+ stormBaseCallback = new ConcurrentHashMap<>();
+ credentialsCallback = new ConcurrentHashMap<>();
+ logConfigCallback = new ConcurrentHashMap<>();
+ blobstoreCallback = new AtomicReference<>();
+
+ stateId = this.clusterState.register(new Callback() {
+
+ public <T> Object execute(T... args) {
+ if (args == null) {
+ LOG.warn("Input args is null");
+ return null;
+ } else if (args.length < 2) {
+ LOG.warn("Input args is invalid, args length:" +
args.length);
+ return null;
+ }
+ String path = (String) args[1];
+
+ List<String> toks = Zookeeper.tokenizePath(path);
+ int size = toks.size();
+ if (size >= 1) {
+ String params = null;
+ String root = toks.get(0);
+ IFn fn = null;
+ if (root.equals(Cluster.ASSIGNMENTS_ROOT)) {
+ if (size == 1) {
+ // set null and get the old value
+ issueCallback(assignmentsCallback);
+ } else {
+ issueMapCallback(assignmentInfoCallback,
toks.get(1));
+ issueMapCallback(assignmentVersionCallback,
toks.get(1));
+
issueMapCallback(assignmentInfoWithVersionCallback, toks.get(1));
+ }
+
+ } else if (root.equals(Cluster.SUPERVISORS_ROOT)) {
+ issueCallback(supervisorsCallback);
+ } else if (root.equals(Cluster.BLOBSTORE_ROOT)) {
+ issueCallback(blobstoreCallback);
+ } else if (root.equals(Cluster.STORMS_ROOT) && size >
1) {
+ issueMapCallback(stormBaseCallback, toks.get(1));
+ } else if (root.equals(Cluster.CREDENTIALS_ROOT) &&
size > 1) {
+ issueMapCallback(credentialsCallback, toks.get(1));
+ } else if (root.equals(Cluster.LOGCONFIG_ROOT) && size
> 1) {
+ issueMapCallback(logConfigCallback, toks.get(1));
+ } else if (root.equals(Cluster.BACKPRESSURE_ROOT) &&
size > 1) {
+ issueMapCallback(logConfigCallback, toks.get(1));
+ } else {
+ LOG.error("{} Unknown callback for subtree {}",
new RuntimeException("Unknown callback for this path"), path);
+ Runtime.getRuntime().exit(30);
+ }
+
+ }
+
+ return null;
+ }
+
+ });
+
+ String[] pathlist = { Cluster.ASSIGNMENTS_SUBTREE,
Cluster.STORMS_SUBTREE, Cluster.SUPERVISORS_SUBTREE,
Cluster.WORKERBEATS_SUBTREE,
+ Cluster.ERRORS_SUBTREE, Cluster.BLOBSTORE_SUBTREE,
Cluster.NIMBUSES_SUBTREE, Cluster.LOGCONFIG_SUBTREE };
+ for (String path : pathlist) {
+ this.clusterState.mkdirs(path, acls);
+ }
+
+ }
+
+ protected void issueCallback(AtomicReference<IFn> cb) {
+ IFn callback = cb.getAndSet(null);
+ if (callback != null)
+ callback.invoke();
+ }
+
+ protected void issueMapCallback(ConcurrentHashMap<String, IFn>
callbackConcurrentHashMap, String key) {
+ IFn callback = callbackConcurrentHashMap.remove(key);
+ if (callback != null)
+ callback.invoke();
+ }
+
+ @Override
+ public List<String> assignments(IFn callback) {
+ if (callback != null) {
+ assignmentsCallback.set(callback);
+ }
+ return clusterState.get_children(Cluster.ASSIGNMENTS_SUBTREE,
callback != null);
+ }
+
+ @Override
+ public Assignment assignmentInfo(String stormId, IFn callback) {
+ if (callback != null) {
+ assignmentInfoCallback.put(stormId, callback);
+ }
+ byte[] serialized =
clusterState.get_data(Cluster.assignmentPath(stormId), callback != null);
+ return Cluster.maybeDeserialize(serialized, Assignment.class);
+ }
+
+ @Override
+ public APersistentMap assignmentInfoWithVersion(String stormId, IFn
callback) {
+ if (callback != null) {
+ assignmentInfoWithVersionCallback.put(stormId, callback);
+ }
+ APersistentMap aPersistentMap =
clusterState.get_data_with_version(Cluster.assignmentPath(stormId), callback !=
null);
+ Assignment assignment = Cluster.maybeDeserialize((byte[])
aPersistentMap.get("data"), Assignment.class);
+ Integer version = (Integer) aPersistentMap.get("version");
+ APersistentMap map = new PersistentArrayMap(new Object[] {
RT.keyword(null, "data"), assignment, RT.keyword(null, "version"), version });
+ return map;
+ }
+
+ @Override
+ public Integer assignmentVersion(String stormId, IFn callback) throws
Exception {
+ if (callback != null) {
+ assignmentVersionCallback.put(stormId, callback);
+ }
+ return clusterState.get_version(Cluster.assignmentPath(stormId),
callback != null);
+ }
+
+ // blobstore state
+ @Override
+ public List<String> blobstoreInfo(String blobKey) {
+ String path = Cluster.blobstorePath(blobKey);
+ clusterState.sync_path(path);
+ return clusterState.get_children(path, false);
+ }
+
+ @Override
+ public List nimbuses() {
+ List<NimbusSummary> nimbusSummaries = new ArrayList<>();
+ List<String> nimbusIds =
clusterState.get_children(Cluster.NIMBUSES_SUBTREE, false);
+ for (String nimbusId : nimbusIds) {
+ byte[] serialized =
clusterState.get_data(Cluster.nimbusPath(nimbusId), false);
+ NimbusSummary nimbusSummary =
Cluster.maybeDeserialize(serialized, NimbusSummary.class);
+ nimbusSummaries.add(nimbusSummary);
+ }
+ return nimbusSummaries;
+ }
+
+ @Override
+ public void addNimbusHost(final String nimbusId, final NimbusSummary
nimbusSummary) {
+ // explicit delete for ephmeral node to ensure this session
creates the entry.
+ clusterState.delete_node(Cluster.nimbusPath(nimbusId));
+ clusterState.add_listener(new ConnectionStateListener() {
+ @Override
+ public void stateChanged(CuratorFramework curatorFramework,
ConnectionState connectionState) {
+ LOG.info("Connection state listener invoked, zookeeper
connection state has changed to {}", connectionState);
+ if (connectionState.equals(ConnectionState.RECONNECTED)) {
+ LOG.info("Connection state has changed to reconnected
so setting nimbuses entry one more time");
+
clusterState.set_ephemeral_node(Cluster.nimbusPath(nimbusId),
Utils.serialize(nimbusSummary), acls);
+ }
+
+ }
+ });
+
+ clusterState.set_ephemeral_node(Cluster.nimbusPath(nimbusId),
Utils.serialize(nimbusSummary), acls);
+ }
+
+ @Override
+ public List<String> activeStorms() {
+ return clusterState.get_children(Cluster.STORMS_SUBTREE, false);
+ }
+
+ @Override
+ public StormBase stormBase(String stormId, IFn callback) {
+ if (callback != null) {
+ stormBaseCallback.put(stormId, callback);
+ }
+ return
Cluster.maybeDeserialize(clusterState.get_data(Cluster.stormPath(stormId),
callback != null), StormBase.class);
+ }
+
+ @Override
+ public ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId,
String node, Long port) {
+ byte[] bytes =
clusterState.get_worker_hb(Cluster.workerbeatPath(stormId, node, port), false);
+ if (bytes != null) {
--- End diff --
this if block is not needed
> port backtype.storm.cluster to java
> -----------------------------------
>
> Key: STORM-1273
> URL: https://issues.apache.org/jira/browse/STORM-1273
> Project: Apache Storm
> Issue Type: New Feature
> Components: storm-core
> Reporter: Robert Joseph Evans
> Assignee: Basti Liu
> Labels: java-migration, jstorm-merger
>
> current state of the cluster (Some of this moves to java as a part of
> heartbeat server)
> https://github.com/apache/storm/tree/jstorm-import/jstorm-core/src/main/java/com/alibaba/jstorm/cluster
> as an example
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)