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 2016/02/24 17:17:11 UTC
[01/27] storm git commit: port org.apache.storm.cluster.cluster.clj
Repository: storm
Updated Branches:
refs/heads/master 11232b539 -> 71d615b7c
port org.apache.storm.cluster.cluster.clj
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/682d31c8
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/682d31c8
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/682d31c8
Branch: refs/heads/master
Commit: 682d31c8a5f53a59f47b4df4bd35ff828e7e0aa5
Parents: 66d7a39
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Mon Feb 1 15:17:49 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Mon Feb 1 15:17:49 2016 +0800
----------------------------------------------------------------------
.../jvm/org/apache/storm/cluster/Cluster.java | 204 ++++++
.../org/apache/storm/cluster/ClusterState.java | 11 +-
.../storm/cluster/DistributedClusterState.java | 269 ++++++++
.../apache/storm/cluster/StormClusterState.java | 129 ++++
.../storm/cluster/StormZkClusterState.java | 662 +++++++++++++++++++
5 files changed, 1271 insertions(+), 4 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/682d31c8/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java b/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java
new file mode 100644
index 0000000..2d6f306
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java
@@ -0,0 +1,204 @@
+/**
+ * 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 org.apache.storm.Config;
+import org.apache.storm.generated.ClusterWorkerHeartbeat;
+import org.apache.storm.generated.ExecutorInfo;
+import org.apache.storm.generated.ExecutorStats;
+import org.apache.storm.generated.ProfileAction;
+import org.apache.storm.utils.Utils;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
+import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLEncoder;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class Cluster {
+
+ public static final String ZK_SEPERATOR = "/";
+
+ public static final String ASSIGNMENTS_ROOT = "assignments";
+ public static final String CODE_ROOT = "code";
+ public static final String STORMS_ROOT = "storms";
+ public static final String SUPERVISORS_ROOT = "supervisors";
+ public static final String WORKERBEATS_ROOT = "workerbeats";
+ public static final String BACKPRESSURE_ROOT = "backpressure";
+ public static final String ERRORS_ROOT = "errors";
+ public static final String BLOBSTORE_ROOT = "blobstore";
+ public static final String BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_ROOT = "blobstoremaxkeysequencenumber";
+ public static final String NIMBUSES_ROOT = "nimbuses";
+ public static final String CREDENTIALS_ROOT = "credentials";
+ public static final String LOGCONFIG_ROOT = "logconfigs";
+ public static final String PROFILERCONFIG_ROOT = "profilerconfigs";
+
+ public static final String ASSIGNMENTS_SUBTREE;
+ public static final String STORMS_SUBTREE;
+ public static final String SUPERVISORS_SUBTREE;
+ public static final String WORKERBEATS_SUBTREE;
+ public static final String BACKPRESSURE_SUBTREE;
+ public static final String ERRORS_SUBTREE;
+ public static final String BLOBSTORE_SUBTREE;
+ public static final String BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_SUBTREE;
+ public static final String NIMBUSES_SUBTREE;
+ public static final String CREDENTIALS_SUBTREE;
+ public static final String LOGCONFIG_SUBTREE;
+ public static final String PROFILERCONFIG_SUBTREE;
+
+ static {
+ ASSIGNMENTS_SUBTREE = ZK_SEPERATOR + ASSIGNMENTS_ROOT;
+ STORMS_SUBTREE = ZK_SEPERATOR + STORMS_ROOT;
+ SUPERVISORS_SUBTREE = ZK_SEPERATOR + SUPERVISORS_ROOT;
+ WORKERBEATS_SUBTREE = ZK_SEPERATOR + WORKERBEATS_ROOT;
+ BACKPRESSURE_SUBTREE = ZK_SEPERATOR + BACKPRESSURE_ROOT;
+ ERRORS_SUBTREE = ZK_SEPERATOR + ERRORS_ROOT;
+ BLOBSTORE_SUBTREE = ZK_SEPERATOR + BLOBSTORE_ROOT;
+ BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_SUBTREE = ZK_SEPERATOR + BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_ROOT;
+ NIMBUSES_SUBTREE = ZK_SEPERATOR + NIMBUSES_ROOT;
+ CREDENTIALS_SUBTREE = ZK_SEPERATOR + CREDENTIALS_ROOT;
+ LOGCONFIG_SUBTREE = ZK_SEPERATOR + LOGCONFIG_ROOT;
+ PROFILERCONFIG_SUBTREE = ZK_SEPERATOR + PROFILERCONFIG_ROOT;
+ }
+
+ public static List<ACL> mkTopoOnlyAcls(Map topoConf) throws NoSuchAlgorithmException {
+ List<ACL> aclList = new ArrayList<>();
+ String payload = (String)topoConf.get(Config.STORM_ZOOKEEPER_AUTH_PAYLOAD);
+ if (Utils.isZkAuthenticationConfiguredStormServer(topoConf)){
+ ACL acl1 = ZooDefs.Ids.CREATOR_ALL_ACL.get(0);
+ aclList.add(acl1);
+ ACL acl2 = new ACL(ZooDefs.Perms.READ, new Id("digest", DigestAuthenticationProvider.generateDigest(payload)));
+ aclList.add(acl2);
+ }
+ return aclList;
+ }
+
+ public static String supervisorPath(String id) {
+ return SUPERVISORS_SUBTREE + ZK_SEPERATOR + id;
+ }
+
+ public static String assignmentPath(String id) {
+ return ASSIGNMENTS_SUBTREE + ZK_SEPERATOR + id;
+ }
+
+ public static String blobstorePath(String key) {
+ return BLOBSTORE_SUBTREE + ZK_SEPERATOR + key;
+ }
+
+ public static String blobstoreMaxKeySequenceNumberPath(String key) {
+ return BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_SUBTREE + ZK_SEPERATOR + key;
+ }
+
+ public static String nimbusPath(String id) {
+ return NIMBUSES_SUBTREE + ZK_SEPERATOR + id;
+ }
+
+ public static String stormPath(String id) {
+ return STORMS_SUBTREE + ZK_SEPERATOR + id;
+ }
+
+ public static String workerbeatStormRoot(String stormId) {
+ return WORKERBEATS_SUBTREE + ZK_SEPERATOR + stormId;
+ }
+
+ public static String workerbeatPath(String stormId, String node, Long port) {
+ return workerbeatStormRoot(stormId) + ZK_SEPERATOR + node + "-" + port;
+ }
+
+ public static String backpressureStormRoot(String stormId) {
+ return BACKPRESSURE_SUBTREE + ZK_SEPERATOR + stormId;
+ }
+
+ public static String backpressurePath(String stormId, String node, Long port) {
+ return backpressureStormRoot(stormId) + ZK_SEPERATOR + node + "-" + port;
+ }
+
+ public static String errorStormRoot(String stormId) {
+ return ERRORS_SUBTREE + ZK_SEPERATOR + stormId;
+ }
+
+ public static String errorPath(String stormId, String componentId) throws UnsupportedEncodingException {
+ return errorStormRoot(stormId) + ZK_SEPERATOR + URLEncoder.encode(componentId, "UTF-8");
+ }
+
+ public static String lastErrorPath(String stormId, String componentId) throws UnsupportedEncodingException {
+ return errorPath(stormId, componentId) + "-last-error";
+ }
+
+ public static String credentialsPath(String stormId) {
+ return CREDENTIALS_SUBTREE + ZK_SEPERATOR + stormId;
+ }
+
+ public static String logConfigPath(String stormId) {
+ return LOGCONFIG_SUBTREE + ZK_SEPERATOR + stormId;
+ }
+
+ public static String profilerConfigPath(String stormId) {
+ return PROFILERCONFIG_SUBTREE + ZK_SEPERATOR + stormId;
+ }
+
+ public static String profilerConfigPath(String stormId, String host, Long port, ProfileAction requestType) {
+ return profilerConfigPath(stormId) + ZK_SEPERATOR + host + "_" + port + "_" + requestType;
+ }
+
+ public static <T> T maybeDeserialize(byte[] serialized, Class<T> clazz){
+ if (serialized != null){
+ return Utils.deserialize(serialized, clazz);
+ }
+ return null;
+ }
+
+ //Ensures that we only return heartbeats for executors assigned to this worker
+ public static Map<ExecutorInfo, ClusterWorkerHeartbeat> convertExecutorBeats(List<ExecutorInfo> executors, ClusterWorkerHeartbeat workerHeartbeat){
+ Map<ExecutorInfo, ClusterWorkerHeartbeat> executorWhb = new HashMap<>();
+ Map<ExecutorInfo, ExecutorStats> executorStatsMap = workerHeartbeat.get_executor_stats();
+ for (ExecutorInfo executor : executors){
+ if(executorStatsMap.containsKey(executor)){
+ executorWhb.put(executor, workerHeartbeat);
+ }
+ }
+ return executorWhb;
+ }
+
+ // TO be remove
+ public static <K, V> HashMap<V, List<K>> reverseMap(Map<K, V> map) {
+ HashMap<V, List<K>> rtn = new HashMap<V, List<K>>();
+ if (map == null) {
+ return rtn;
+ }
+ for (Map.Entry<K, V> entry : map.entrySet()) {
+ K key = entry.getKey();
+ V val = entry.getValue();
+ List<K> list = rtn.get(val);
+ if (list == null) {
+ list = new ArrayList<K>();
+ rtn.put(entry.getValue(), list);
+ }
+ list.add(key);
+ }
+ return rtn;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/682d31c8/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java
index fdac92c..51e42ff 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java
@@ -20,6 +20,9 @@ package org.apache.storm.cluster;
import clojure.lang.APersistentMap;
import clojure.lang.IFn;
import java.util.List;
+
+import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.storm.callback.Callback;
import org.apache.zookeeper.data.ACL;
/**
@@ -47,7 +50,7 @@ public interface ClusterState {
* @return is an id that can be passed to unregister(...) to unregister the
* callback.
*/
- String register(IFn callback);
+ String register(Callback callback);
/**
* Unregisters a callback function that was registered with register(...).
@@ -73,7 +76,7 @@ public interface ClusterState {
* @param acls The acls to apply to the path. May be null.
* @return path
*/
- String mkdirs(String path, List<ACL> acls);
+ void mkdirs(String path, List<ACL> acls);
/**
* Deletes the node at a given path, and any child nodes that may exist.
@@ -99,7 +102,7 @@ public interface ClusterState {
* register method. Very useful for catching updates to nodes.
* @return The integer version of this node.
*/
- Integer get_version(String path, boolean watch);
+ Integer get_version(String path, boolean watch) throws Exception;
/**
* Check if a node exists and optionally set a watch on the path.
@@ -197,7 +200,7 @@ public interface ClusterState {
* @param listener A ClusterStateListener to handle changing cluster state
* events.
*/
- void add_listener(ClusterStateListener listener);
+ void add_listener(final ConnectionStateListener listener);
/**
* Force consistency on a path. Any writes committed on the path before
http://git-wip-us.apache.org/repos/asf/storm/blob/682d31c8/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java
new file mode 100644
index 0000000..3e0beb1
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java
@@ -0,0 +1,269 @@
+/**
+ * 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 org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.state.*;
+import org.apache.curator.framework.state.ConnectionState;
+import org.apache.storm.Config;
+import org.apache.storm.callback.Callback;
+import org.apache.storm.callback.WatcherCallBack;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.zookeeper.Zookeeper;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class DistributedClusterState implements ClusterState {
+
+ private static Logger LOG = LoggerFactory.getLogger(DistributedClusterState.class);
+
+ private ConcurrentHashMap<String, Callback> callbacks = new ConcurrentHashMap<String, Callback>();
+ private CuratorFramework zkWriter;
+ private CuratorFramework zkReader;
+ private AtomicBoolean active;
+
+ private boolean isNimbus;
+ private Map authConf;
+ private Map<Object, Object> conf;
+
+ public DistributedClusterState(Map<Object, Object> conf, Map authConf, List<ACL> acls, ClusterStateContext context) throws Exception {
+ this.conf = conf;
+ this.authConf = authConf;
+ if (context.getDaemonType().equals(DaemonType.NIMBUS))
+ this.isNimbus = true;
+
+ // just mkdir STORM_ZOOKEEPER_ROOT dir
+ CuratorFramework zkTemp = mkZk();
+ String rootPath = String.valueOf(conf.get(Config.STORM_ZOOKEEPER_ROOT));
+ Zookeeper.mkdirs(zkTemp, rootPath, acls);
+ zkTemp.close();
+
+ active = new AtomicBoolean(true);
+ zkWriter = mkZk(new WatcherCallBack() {
+ @Override
+ public void execute(Watcher.Event.KeeperState state, Watcher.Event.EventType type, String path) {
+ if (active.get()) {
+ if (!(state.equals(Watcher.Event.KeeperState.SyncConnected))) {
+ LOG.warn("Received event {} : {}: {} with disconnected Zookeeper.", state, type, path);
+ } else {
+ LOG.info("Received event {} : {} : {}", state, type, path);
+ }
+
+ if (!type.equals(Watcher.Event.EventType.None)) {
+ for (Map.Entry<String, Callback> e : callbacks.entrySet()) {
+ Callback fn = e.getValue();
+ fn.execute(type, path);
+ }
+ }
+ }
+ }
+ });
+ if (isNimbus) {
+ zkReader = mkZk(new WatcherCallBack() {
+ @Override
+ public void execute(Watcher.Event.KeeperState state, Watcher.Event.EventType type, String path) {
+ if (active.get()) {
+ if (!(state.equals(Watcher.Event.KeeperState.SyncConnected))) {
+ LOG.warn("Received event {} : {}: {} with disconnected Zookeeper.", state, type, path);
+ } else {
+ LOG.info("Received event {} : {} : {}", state, type, path);
+ }
+
+ if (!type.equals(Watcher.Event.EventType.None)) {
+ for (Map.Entry<String, Callback> e : callbacks.entrySet()) {
+ Callback fn = e.getValue();
+ fn.execute(type, path);
+ }
+ }
+ }
+ }
+ });
+ } else {
+ zkReader = zkWriter;
+ }
+
+ }
+
+ @SuppressWarnings("unchecked")
+ private CuratorFramework mkZk() throws IOException {
+ return Zookeeper.mkClient(conf, (List<String>) conf.get(Config.STORM_ZOOKEEPER_SERVERS), conf.get(Config.STORM_ZOOKEEPER_PORT), "", authConf);
+ }
+
+ @SuppressWarnings("unchecked")
+ private CuratorFramework mkZk(WatcherCallBack watcher) throws NumberFormatException, IOException {
+ return Zookeeper.mkClient(conf, (List<String>) conf.get(Config.STORM_ZOOKEEPER_SERVERS), conf.get(Config.STORM_ZOOKEEPER_PORT),
+ String.valueOf(conf.get(Config.STORM_ZOOKEEPER_ROOT)), watcher, authConf);
+ }
+
+ @Override
+ public void delete_node_blobstore(String path, String nimbusHostPortInfo) {
+
+ }
+
+ @Override
+ public String register(Callback callback) {
+ String id = UUID.randomUUID().toString();
+ this.callbacks.put(id, callback);
+ return id;
+ }
+
+ @Override
+ public void unregister(String id) {
+ this.callbacks.remove(id);
+ }
+
+ @Override
+ public String create_sequential(String path, byte[] data, List<ACL> acls) {
+ return Zookeeper.createNode(zkWriter, path, data, CreateMode.EPHEMERAL_SEQUENTIAL, acls);
+ }
+
+ @Override
+ public void mkdirs(String path, List<ACL> acls) {
+ Zookeeper.mkdirs(zkWriter, path, acls);
+ }
+
+ @Override
+ public void delete_node(String path) {
+ Zookeeper.deleteNode(zkWriter, path);
+ }
+
+ @Override
+ public void set_ephemeral_node(String path, byte[] data, List<ACL> acls) {
+ Zookeeper.mkdirs(zkWriter, parentPath(path), acls);
+ if (Zookeeper.exists(zkWriter, path, false)) {
+ try {
+ Zookeeper.setData(zkWriter, path, data);
+ } catch (RuntimeException e) {
+ if (Utils.exceptionCauseIsInstanceOf(KeeperException.NodeExistsException.class, e)) {
+ Zookeeper.createNode(zkWriter, path, data, CreateMode.EPHEMERAL, acls);
+ } else {
+ throw e;
+ }
+ }
+
+ } else {
+ Zookeeper.createNode(zkWriter, path, data, CreateMode.EPHEMERAL, acls);
+ }
+ }
+
+ @Override
+ public Integer get_version(String path, boolean watch) throws Exception {
+ Integer ret = Zookeeper.getVersion(zkReader, path, watch);
+ return ret;
+ }
+
+ @Override
+ public boolean node_exists(String path, boolean watch) {
+ return Zookeeper.existsNode(zkWriter, path, watch);
+ }
+
+ @Override
+ public List<String> get_children(String path, boolean watch) {
+ return Zookeeper.getChildren(zkReader, path, watch);
+ }
+
+ @Override
+ public void close() {
+ this.active.set(false);
+ zkWriter.close();
+ if (isNimbus) {
+ zkReader.close();
+ }
+ }
+
+ @Override
+ public void set_data(String path, byte[] data, List<ACL> acls) {
+ if (Zookeeper.exists(zkWriter, path, false)) {
+ Zookeeper.setData(zkWriter, path, data);
+ } else {
+ Zookeeper.mkdirs(zkWriter, parentPath(path), acls);
+ Zookeeper.createNode(zkWriter, path, data, CreateMode.PERSISTENT, acls);
+ }
+ }
+
+ @Override
+ public byte[] get_data(String path, boolean watch) {
+ byte[] ret = null;
+
+ ret = Zookeeper.getData(zkReader, path, watch);
+
+ return ret;
+ }
+
+ @Override
+ public APersistentMap get_data_with_version(String path, boolean watch) {
+ return Zookeeper.getDataWithVersion(zkReader, path, watch);
+ }
+
+ @Override
+ public void set_worker_hb(String path, byte[] data, List<ACL> acls) {
+ set_data(path, data, acls);
+ }
+
+ @Override
+ public byte[] get_worker_hb(String path, boolean watch) {
+ return Zookeeper.getData(zkReader, path, watch);
+ }
+
+ @Override
+ public List<String> get_worker_hb_children(String path, boolean watch) {
+ return get_children(path, watch);
+ }
+
+ @Override
+ public void delete_worker_hb(String path) {
+ delete_node(path);
+ }
+
+ @Override
+ public void add_listener(final ConnectionStateListener listener) {
+ Zookeeper.addListener(zkReader, new ConnectionStateListener() {
+ @Override
+ public void stateChanged(CuratorFramework curatorFramework, ConnectionState connectionState) {
+ listener.stateChanged(curatorFramework, connectionState);
+ }
+ });
+ }
+
+ @Override
+ public void sync_path(String path) {
+ Zookeeper.syncPath(zkWriter, path);
+ }
+
+ // To be remove when finished port Util.clj
+ public static String parentPath(String path) {
+ List<String> toks = Zookeeper.tokenizePath(path);
+ int size = toks.size();
+ if (size > 0) {
+ toks.remove(size - 1);
+ }
+ return Zookeeper.toksToPath(toks);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/682d31c8/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
new file mode 100644
index 0000000..b3c0f90
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
@@ -0,0 +1,129 @@
+/**
+ * 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 org.apache.storm.callback.Callback;
+import org.apache.storm.generated.*;
+import org.apache.storm.nimbus.NimbusInfo;
+
+import java.security.NoSuchAlgorithmException;
+import java.util.List;
+import java.util.Map;
+
+public interface StormClusterState {
+ public List<String> assignments(Callback callback);
+
+ public Assignment assignmentInfo(String stormId, Callback callback);
+
+ public APersistentMap assignmentInfoWithVersion(String stormId, Callback callback);
+
+ public Integer assignmentVersion(String stormId, Callback callback) throws Exception;
+
+ // returns key information under /storm/blobstore/key
+ public List<String> blobstoreInfo(String blobKey);
+
+ // returns list of nimbus summaries stored under /stormroot/nimbuses/<nimbus-ids> -> <data>
+ public List nimbuses();
+
+ // adds the NimbusSummary to /stormroot/nimbuses/nimbus-id
+ public void addNimbusHost(String nimbusId, NimbusSummary nimbusSummary);
+
+ public List<String> activeStorms();
+
+ public StormBase stormBase(String stormId, Callback callback);
+
+ public ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, String node, Long port);
+
+ public List<ProfileRequest> getWorkerProfileRequets(String stormId, NodeInfo nodeInfo, boolean isThrift);
+
+ public List<ProfileRequest> getTopologyProfileRequets(String stormId, boolean isThrift);
+
+ public void setWorkerProfileRequests(String stormId, ProfileRequest profileRequest);
+
+ public void deleteTopologyProfileRequests(String stormId, ProfileRequest profileRequest);
+
+ public Map<ExecutorInfo, ClusterWorkerHeartbeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort);
+
+ public List<String> supervisors(Callback callback);
+
+ public SupervisorInfo supervisorInfo(String supervisorId); // returns nil if doesn't exist
+
+ public void setupHeatbeats(String stormId);
+
+ public void teardownHeatbeats(String stormId);
+
+ public void teardownTopologyErrors(String stormId);
+
+ public List<String> heartbeatStorms();
+
+ public List<String> errorTopologies();
+
+ public void setTopologyLogConfig(String stormId, LogConfig logConfig);
+
+ public LogConfig topologyLogConfig(String stormId, Callback cb);
+
+ public void workerHeartbeat(String stormId, String node, Long port, ClusterWorkerHeartbeat info);
+
+ public void removeWorkerHeartbeat(String stormId, String node, Long port);
+
+ public void supervisorHeartbeat(String supervisorId, SupervisorInfo info);
+
+ public void workerBackpressure(String stormId, String node, Long port, boolean on);
+
+ public boolean topologyBackpressure(String stormId, Callback callback);
+
+ public void setupBackpressure(String stormId);
+
+ public void removeWorkerBackpressure(String stormId, String node, Long port);
+
+ public void activateStorm(String stormId, StormBase stormBase);
+
+ public void updateStorm(String stormId, StormBase newElems);
+
+ public void removeStormBase(String stormId);
+
+ public void setAssignment(String stormId, Assignment info);
+
+ // sets up information related to key consisting of nimbus
+ // host:port and version info of the blob
+ public void setupBlobstore(String key, NimbusInfo nimbusInfo, String versionInfo);
+
+ public List<String> activeKeys();
+
+ public List<String> blobstore(Callback callback);
+
+ public void removeStorm(String stormId);
+
+ public void removeBlobstoreKey(String blobKey);
+
+ public void removeKeyVersion(String blobKey);
+
+ public void reportError(String stormId, String componentId, String node, Long port, String error);
+
+ public List<ErrorInfo> errors(String stormId, String componentId);
+
+ public ErrorInfo lastError(String stormId, String componentId);
+
+ public void setCredentials(String stormId, Credentials creds, Map topoConf) throws NoSuchAlgorithmException;
+
+ public Credentials credentials(String stormId, Callback callback);
+
+ public void disconnect();
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/682d31c8/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
new file mode 100644
index 0000000..93d29b2
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
@@ -0,0 +1,662 @@
+/**
+ * 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.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, Callback> assignmentInfoCallback;
+ private ConcurrentHashMap<String, Callback> assignmentInfoWithVersionCallback;
+ private ConcurrentHashMap<String, Callback> assignmentVersionCallback;
+ private AtomicReference<Callback> supervisorsCallback;
+ // we want to reigister a topo directory getChildren callback for all workers of this dir
+ private ConcurrentHashMap<String, Callback> backPressureCallback;
+ private AtomicReference<Callback> assignmentsCallback;
+ private ConcurrentHashMap<String, Callback> stormBaseCallback;
+ private AtomicReference<Callback> blobstoreCallback;
+ private ConcurrentHashMap<String, Callback> credentialsCallback;
+ private ConcurrentHashMap<String, Callback> 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);
+ Callback 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<Callback> cb) {
+ Callback callback = cb.getAndSet(null);
+ callback.execute();
+ }
+
+ protected void issueMapCallback(ConcurrentHashMap<String, Callback> callbackConcurrentHashMap, String key) {
+ Callback callback = callbackConcurrentHashMap.remove(key);
+ callback.execute();
+ }
+
+ @Override
+ public List<String> assignments(Callback callback) {
+ if (callback != null) {
+ assignmentsCallback.set(callback);
+ }
+ return clusterState.get_children(Cluster.ASSIGNMENTS_SUBTREE, callback != null);
+ }
+
+ @Override
+ public Assignment assignmentInfo(String stormId, Callback 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, Callback 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, Callback 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, Callback 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) {
+ return Cluster.maybeDeserialize(bytes, ClusterWorkerHeartbeat.class);
+ }
+ return null;
+ }
+
+ @Override
+ public List<ProfileRequest> getWorkerProfileRequets(String stormId, NodeInfo nodeInfo, boolean isThrift) {
+ List<ProfileRequest> requests = new ArrayList<>();
+ List<ProfileRequest> profileRequests = getTopologyProfileRequets(stormId, isThrift);
+ for (ProfileRequest profileRequest : profileRequests) {
+ NodeInfo nodeInfo1 = profileRequest.get_nodeInfo();
+ if (nodeInfo1.equals(nodeInfo))
+ requests.add(profileRequest);
+ }
+ return requests;
+ }
+
+ @Override
+ public List<ProfileRequest> getTopologyProfileRequets(String stormId, boolean isThrift) {
+ List<ProfileRequest> profileRequests = new ArrayList<>();
+ String path = Cluster.profilerConfigPath(stormId);
+ if (clusterState.node_exists(path, false)) {
+ List<String> strs = clusterState.get_children(path, false);
+ for (String str : strs) {
+ String childPath = path + Cluster.ZK_SEPERATOR + str;
+ byte[] raw = clusterState.get_data(childPath, false);
+ ProfileRequest request = Cluster.maybeDeserialize(raw, ProfileRequest.class);
+ if (request != null)
+ profileRequests.add(request);
+ }
+ }
+ return profileRequests;
+ }
+
+ @Override
+ public void setWorkerProfileRequests(String stormId, ProfileRequest profileRequest) {
+ ProfileAction profileAction = profileRequest.get_action();
+ String host = profileRequest.get_nodeInfo().get_node();
+ Long port = profileRequest.get_nodeInfo().get_port_iterator().next();
+ String path = Cluster.profilerConfigPath(stormId, host, port, profileAction);
+ clusterState.set_data(path, Utils.serialize(profileRequest), acls);
+ }
+
+ @Override
+ public void deleteTopologyProfileRequests(String stormId, ProfileRequest profileRequest) {
+ ProfileAction profileAction = profileRequest.get_action();
+ String host = profileRequest.get_nodeInfo().get_node();
+ Long port = profileRequest.get_nodeInfo().get_port_iterator().next();
+ String path = Cluster.profilerConfigPath(stormId, host, port, profileAction);
+ clusterState.delete_node(path);
+ }
+
+ @Override
+ public Map<ExecutorInfo, ClusterWorkerHeartbeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort) {
+ Map<ExecutorInfo, ClusterWorkerHeartbeat> executorWhbs = new HashMap<>();
+
+ Map<NodeInfo, List<List<Long>>> nodePortExecutors = Cluster.reverseMap(executorNodePort);
+ for (Map.Entry<NodeInfo, List<List<Long>>> entry : nodePortExecutors.entrySet()) {
+
+ String node = entry.getKey().get_node();
+ Long port = entry.getKey().get_port_iterator().next();
+ ClusterWorkerHeartbeat whb = getWorkerHeartbeat(stormId, node, port);
+ List<ExecutorInfo> executorInfoList = new ArrayList<>();
+ for (List<Long> list : entry.getValue()) {
+ executorInfoList.add(new ExecutorInfo(list.get(0).intValue(), list.get(list.size() - 1).intValue()));
+ }
+ executorWhbs.putAll(Cluster.convertExecutorBeats(executorInfoList, whb));
+ }
+ return executorWhbs;
+ }
+
+ @Override
+ public List<String> supervisors(Callback callback) {
+ if (callback != null) {
+ supervisorsCallback.set(callback);
+ }
+ return clusterState.get_children(Cluster.SUPERVISORS_SUBTREE, callback != null);
+ }
+
+ @Override
+ public SupervisorInfo supervisorInfo(String supervisorId) {
+ String path = Cluster.supervisorPath(supervisorId);
+ return Cluster.maybeDeserialize(clusterState.get_data(path, false), SupervisorInfo.class);
+ }
+
+ @Override
+ public void setupHeatbeats(String stormId) {
+ clusterState.mkdirs(Cluster.workerbeatStormRoot(stormId), acls);
+ }
+
+ @Override
+ public void teardownHeatbeats(String stormId) {
+ try {
+ clusterState.delete_worker_hb(Cluster.workerbeatStormRoot(stormId));
+ } catch (Exception e) {
+ if (Zookeeper.exceptionCause(KeeperException.class, e)) {
+ // do nothing
+ LOG.warn("Could not teardown heartbeats for {}.", stormId);
+ } else {
+ throw e;
+ }
+ }
+ }
+
+ @Override
+ public void teardownTopologyErrors(String stormId) {
+ try {
+ clusterState.delete_node(Cluster.errorStormRoot(stormId));
+ } catch (Exception e) {
+ if (Zookeeper.exceptionCause(KeeperException.class, e)) {
+ // do nothing
+ LOG.warn("Could not teardown errors for {}.", stormId);
+ } else {
+ throw e;
+ }
+ }
+ }
+
+ @Override
+ public List<String> heartbeatStorms() {
+ return clusterState.get_worker_hb_children(Cluster.WORKERBEATS_SUBTREE, false);
+ }
+
+ @Override
+ public List<String> errorTopologies() {
+ return clusterState.get_children(Cluster.ERRORS_SUBTREE, false);
+ }
+
+ @Override
+ public void setTopologyLogConfig(String stormId, LogConfig logConfig) {
+ clusterState.set_data(Cluster.logConfigPath(stormId), Utils.serialize(logConfig), acls);
+ }
+
+ @Override
+ public LogConfig topologyLogConfig(String stormId, Callback cb) {
+ String path = Cluster.logConfigPath(stormId);
+ return Cluster.maybeDeserialize(clusterState.get_data(path, cb != null), LogConfig.class);
+ }
+
+ @Override
+ public void workerHeartbeat(String stormId, String node, Long port, ClusterWorkerHeartbeat info) {
+ if (info != null) {
+ String path = Cluster.workerbeatPath(stormId, node, port);
+ clusterState.set_worker_hb(path, Utils.serialize(info), acls);
+ }
+ }
+
+ @Override
+ public void removeWorkerHeartbeat(String stormId, String node, Long port) {
+ String path = Cluster.workerbeatPath(stormId, node, port);
+ clusterState.delete_worker_hb(path);
+ }
+
+ @Override
+ public void supervisorHeartbeat(String supervisorId, SupervisorInfo info) {
+ String path = Cluster.supervisorPath(supervisorId);
+ clusterState.set_ephemeral_node(path, Utils.serialize(info), acls);
+ }
+
+ // if znode exists and to be not on?, delete; if exists and on?, do nothing;
+ // if not exists and to be on?, create; if not exists and not on?, do nothing;
+ @Override
+ public void workerBackpressure(String stormId, String node, Long port, boolean on) {
+ String path = Cluster.backpressurePath(stormId, node, port);
+ boolean existed = clusterState.node_exists(path, false);
+ if (existed) {
+ if (on == false)
+ clusterState.delete_node(path);
+
+ } else {
+ if (on == true) {
+ clusterState.set_ephemeral_node(path, null, acls);
+ }
+ }
+ }
+
+ // if the backpresure/storm-id dir is empty, this topology has throttle-on, otherwise not.
+ @Override
+ public boolean topologyBackpressure(String stormId, Callback callback) {
+ if (callback != null) {
+ backPressureCallback.put(stormId, callback);
+ }
+ String path = Cluster.backpressureStormRoot(stormId);
+ List<String> childrens = clusterState.get_children(path, callback != null);
+ return childrens.size() > 0;
+
+ }
+
+ @Override
+ public void setupBackpressure(String stormId) {
+ clusterState.mkdirs(Cluster.backpressureStormRoot(stormId), acls);
+ }
+
+ @Override
+ public void removeWorkerBackpressure(String stormId, String node, Long port) {
+ clusterState.delete_node(Cluster.backpressurePath(stormId, node, port));
+ }
+
+ @Override
+ public void activateStorm(String stormId, StormBase stormBase) {
+ String path = Cluster.stormPath(stormId);
+ clusterState.set_data(path, Utils.serialize(stormBase), acls);
+ }
+
+ // maybe exit some questions for updateStorm
+ @Override
+ public void updateStorm(String stormId, StormBase newElems) {
+
+ StormBase stormBase = stormBase(stormId, null);
+ if (stormBase.get_component_executors() != null) {
+ Map<String, Integer> componentExecutors = newElems.get_component_executors();
+ if (componentExecutors == null) {
+ componentExecutors = new HashMap<>();
+ }
+ for (Map.Entry<String, Integer> entry : stormBase.get_component_executors().entrySet()) {
+ if (!componentExecutors.containsKey(entry.getKey())) {
+ componentExecutors.put(entry.getKey(), entry.getValue());
+ }
+ }
+ if (componentExecutors.size() > 0)
+ newElems.set_component_executors(componentExecutors);
+ }
+
+ Map<String, DebugOptions> ComponentDebug = new HashMap<>();
+ Map<String, DebugOptions> oldComponentDebug = stormBase.get_component_debug();
+ if (oldComponentDebug == null)
+ oldComponentDebug = new HashMap<>();
+ Map<String, DebugOptions> newComponentDebug = newElems.get_component_debug();
+ if (newComponentDebug == null)
+ newComponentDebug = new HashMap<>();
+ Set<String> debugOptionsKeys = oldComponentDebug.keySet();
+ debugOptionsKeys.addAll(newComponentDebug.keySet());
+ for (String key : debugOptionsKeys) {
+ boolean enable = false;
+ double samplingpct = 0;
+ if (oldComponentDebug.containsKey(key)) {
+ enable = oldComponentDebug.get(key).is_enable();
+ samplingpct = oldComponentDebug.get(key).get_samplingpct();
+ }
+ if (newComponentDebug.containsKey(key)) {
+ enable = newComponentDebug.get(key).is_enable();
+ samplingpct += newComponentDebug.get(key).get_samplingpct();
+ }
+ DebugOptions debugOptions = new DebugOptions();
+ debugOptions.set_enable(enable);
+ debugOptions.set_samplingpct(samplingpct);
+ ComponentDebug.put(key, debugOptions);
+ }
+ if (ComponentDebug.size() > 0) {
+ newElems.set_component_debug(ComponentDebug);
+ }
+ // only merge some parameters which are optional
+ if (newElems.get_launch_time_secs() == 0) {
+ newElems.set_launch_time_secs(stormBase.get_launch_time_secs());
+ }
+ if (StringUtils.isBlank(newElems.get_owner())) {
+ newElems.set_owner(stormBase.get_owner());
+ }
+ if (newElems.get_topology_action_options() == null) {
+ newElems.set_topology_action_options(stormBase.get_topology_action_options());
+ }
+ if (newElems.get_status() == null) {
+ newElems.set_status(stormBase.get_status());
+ }
+ clusterState.set_data(Cluster.stormPath(stormId), Utils.serialize(newElems), acls);
+ }
+
+ @Override
+ public void removeStormBase(String stormId) {
+ clusterState.delete_node(Cluster.stormPath(stormId));
+ }
+
+ @Override
+ public void setAssignment(String stormId, Assignment info) {
+ clusterState.set_data(Cluster.assignmentPath(stormId), Utils.serialize(info), acls);
+ }
+
+ @Override
+ public void setupBlobstore(String key, NimbusInfo nimbusInfo, String versionInfo) {
+ String path = Cluster.blobstorePath(key) + Cluster.ZK_SEPERATOR + nimbusInfo.toHostPortString() + "_" + versionInfo;
+ LOG.info("set-path: {}", path);
+ clusterState.mkdirs(Cluster.blobstorePath(key), acls);
+ clusterState.delete_node_blobstore(Cluster.blobstorePath(key), nimbusInfo.toHostPortString());
+ clusterState.set_ephemeral_node(path, null, acls);
+ }
+
+ @Override
+ public List<String> activeKeys() {
+ return clusterState.get_children(Cluster.BLOBSTORE_SUBTREE, false);
+ }
+
+ // blobstore state
+ @Override
+ public List<String> blobstore(Callback callback) {
+ if (callback != null) {
+ blobstoreCallback.set(callback);
+ }
+ clusterState.sync_path(Cluster.BLOBSTORE_SUBTREE);
+ return clusterState.get_children(Cluster.BLOBSTORE_SUBTREE, callback != null);
+
+ }
+
+ @Override
+ public void removeStorm(String stormId) {
+ clusterState.delete_node(Cluster.assignmentPath(stormId));
+ clusterState.delete_node(Cluster.credentialsPath(stormId));
+ clusterState.delete_node(Cluster.logConfigPath(stormId));
+ clusterState.delete_node(Cluster.profilerConfigPath(stormId));
+ removeStormBase(stormId);
+ }
+
+ @Override
+ public void removeBlobstoreKey(String blobKey) {
+ LOG.debug("remove key {}", blobKey);
+ clusterState.delete_node(Cluster.blobstorePath(blobKey));
+ }
+
+ @Override
+ public void removeKeyVersion(String blobKey) {
+ clusterState.delete_node(Cluster.blobstoreMaxKeySequenceNumberPath(blobKey));
+ }
+
+ @Override
+ public void reportError(String stormId, String componentId, String node, Long port, String error) {
+
+ try {
+ String path = Cluster.errorPath(stormId, componentId);
+ String lastErrorPath = Cluster.lastErrorPath(stormId, componentId);
+ ErrorInfo errorInfo = new ErrorInfo(error, Time.currentTimeSecs());
+ errorInfo.set_host(node);
+ errorInfo.set_port(port.intValue());
+ byte[] serData = Utils.serialize(errorInfo);
+ clusterState.mkdirs(path, acls);
+ clusterState.create_sequential(path + Cluster.ZK_SEPERATOR + "e", serData, acls);
+ clusterState.set_data(lastErrorPath, serData, acls);
+ List<String> childrens = clusterState.get_children(path, false);
+
+ Collections.sort(childrens);
+
+ while (childrens.size() >= 10) {
+ clusterState.delete_node(path + Cluster.ZK_SEPERATOR + childrens.remove(0));
+ }
+ } catch (UnsupportedEncodingException e) {
+ throw Utils.wrapInRuntime(e);
+ }
+ }
+
+ @Override
+ public List<ErrorInfo> errors(String stormId, String componentId) {
+ List<ErrorInfo> errorInfos = new ArrayList<>();
+ try {
+ String path = Cluster.errorPath(stormId, componentId);
+ if (clusterState.node_exists(path, false)) {
+ List<String> childrens = clusterState.get_children(path, false);
+ for (String child : childrens) {
+ String childPath = path + Cluster.ZK_SEPERATOR + child;
+ ErrorInfo errorInfo = Cluster.maybeDeserialize(clusterState.get_data(childPath, false), ErrorInfo.class);
+ if (errorInfo != null)
+ errorInfos.add(errorInfo);
+ }
+ }
+ Collections.sort(errorInfos, new Comparator<ErrorInfo>() {
+ public int compare(ErrorInfo arg0, ErrorInfo arg1) {
+ return Integer.compare(arg0.get_error_time_secs(), arg1.get_error_time_secs());
+ }
+ });
+ } catch (Exception e) {
+ throw Utils.wrapInRuntime(e);
+ }
+
+ return errorInfos;
+ }
+
+ @Override
+ public ErrorInfo lastError(String stormId, String componentId) {
+ try {
+ String path = Cluster.lastErrorPath(stormId, componentId);
+ if (clusterState.node_exists(path, false)) {
+ ErrorInfo errorInfo = Cluster.maybeDeserialize(clusterState.get_data(path, false), ErrorInfo.class);
+ return errorInfo;
+ }
+ } catch (UnsupportedEncodingException e) {
+ throw Utils.wrapInRuntime(e);
+ }
+ return null;
+ }
+
+ @Override
+ public void setCredentials(String stormId, Credentials creds, Map topoConf) throws NoSuchAlgorithmException {
+ List<ACL> aclList = Cluster.mkTopoOnlyAcls(topoConf);
+ String path = Cluster.credentialsPath(stormId);
+ clusterState.set_data(path, Utils.serialize(creds), aclList);
+
+ }
+
+ @Override
+ public Credentials credentials(String stormId, Callback callback) {
+ if (callback != null) {
+ credentialsCallback.put(stormId, callback);
+ }
+ String path = Cluster.credentialsPath(stormId);
+ return Cluster.maybeDeserialize(clusterState.get_data(path, callback != null), Credentials.class);
+
+ }
+
+ @Override
+ public void disconnect() {
+ clusterState.unregister(stateId);
+ if (solo)
+ clusterState.close();
+ }
+}
[12/27] storm git commit: Merge branch 'master' into ClusterUtils
Posted by bo...@apache.org.
Merge branch 'master' into ClusterUtils
Conflicts:
storm-core/src/clj/org/apache/storm/cluster.clj
storm-core/src/clj/org/apache/storm/cluster_state/zookeeper_state_factory.clj
storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
storm-core/src/clj/org/apache/storm/daemon/common.clj
storm-core/src/clj/org/apache/storm/daemon/executor.clj
storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
storm-core/src/clj/org/apache/storm/stats.clj
storm-core/src/clj/org/apache/storm/testing.clj
storm-core/src/clj/org/apache/storm/thrift.clj
storm-core/src/clj/org/apache/storm/util.clj
storm-core/src/clj/org/apache/storm/zookeeper.clj
storm-core/test/clj/integration/org/apache/storm/integration_test.clj
storm-core/test/clj/org/apache/storm/cluster_test.clj
storm-core/test/clj/org/apache/storm/nimbus_test.clj
storm-core/test/clj/org/apache/storm/supervisor_test.clj
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/9a8962de
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/9a8962de
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/9a8962de
Branch: refs/heads/master
Commit: 9a8962de9c80fc6a5388fd6d63ab225268530adf
Parents: 2ee8bec 12ceb09
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Mon Feb 15 15:13:33 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Mon Feb 15 22:15:21 2016 +0800
----------------------------------------------------------------------
CHANGELOG.md | 16 +
README.markdown | 1 +
bin/storm-config.cmd | 6 +-
bin/storm.cmd | 47 +-
bin/storm.py | 8 +-
conf/defaults.yaml | 4 +
dev-tools/travis/travis-script.sh | 4 +-
.../starter/trident/TridentMapExample.java | 123 +++
external/sql/storm-sql-core/pom.xml | 9 +
external/storm-elasticsearch/pom.xml | 2 +
.../storm/hbase/security/HBaseSecurityUtil.java | 36 +-
.../jvm/org/apache/storm/kafka/KafkaSpout.java | 8 +-
.../jvm/org/apache/storm/kafka/KafkaUtils.java | 44 +-
.../apache/storm/kafka/PartitionManager.java | 42 +-
.../kafka/trident/TridentKafkaEmitter.java | 23 +-
external/storm-mqtt/core/pom.xml | 4 +-
log4j2/cluster.xml | 2 +-
log4j2/worker.xml | 2 +-
pom.xml | 9 +-
storm-core/pom.xml | 11 +-
.../src/clj/org/apache/storm/LocalCluster.clj | 4 +-
storm-core/src/clj/org/apache/storm/clojure.clj | 8 +-
.../clj/org/apache/storm/command/blobstore.clj | 11 +-
.../org/apache/storm/command/config_value.clj | 25 -
.../org/apache/storm/command/dev_zookeeper.clj | 6 +-
.../clj/org/apache/storm/command/get_errors.clj | 12 +-
.../apache/storm/command/shell_submission.clj | 4 +-
storm-core/src/clj/org/apache/storm/config.clj | 18 +-
.../src/clj/org/apache/storm/converter.clj | 14 +-
.../src/clj/org/apache/storm/daemon/acker.clj | 21 +-
.../src/clj/org/apache/storm/daemon/common.clj | 46 +-
.../src/clj/org/apache/storm/daemon/drpc.clj | 25 +-
.../clj/org/apache/storm/daemon/executor.clj | 530 +++++-----
.../clj/org/apache/storm/daemon/logviewer.clj | 70 +-
.../src/clj/org/apache/storm/daemon/nimbus.clj | 168 ++--
.../clj/org/apache/storm/daemon/supervisor.clj | 205 ++--
.../src/clj/org/apache/storm/daemon/task.clj | 2 +-
.../src/clj/org/apache/storm/daemon/worker.clj | 78 +-
.../src/clj/org/apache/storm/disruptor.clj | 10 +-
storm-core/src/clj/org/apache/storm/event.clj | 2 +-
.../src/clj/org/apache/storm/local_state.clj | 9 +-
.../clj/org/apache/storm/messaging/loader.clj | 34 -
.../clj/org/apache/storm/messaging/local.clj | 23 -
.../org/apache/storm/pacemaker/pacemaker.clj | 7 +-
.../storm/pacemaker/pacemaker_state_factory.clj | 122 ---
.../clj/org/apache/storm/process_simulator.clj | 4 +-
.../apache/storm/scheduler/DefaultScheduler.clj | 7 +-
.../apache/storm/scheduler/EvenScheduler.clj | 23 +-
.../storm/scheduler/IsolationScheduler.clj | 29 +-
storm-core/src/clj/org/apache/storm/stats.clj | 82 +-
storm-core/src/clj/org/apache/storm/testing.clj | 89 +-
storm-core/src/clj/org/apache/storm/thrift.clj | 6 +-
storm-core/src/clj/org/apache/storm/timer.clj | 12 +-
.../clj/org/apache/storm/trident/testing.clj | 9 +-
storm-core/src/clj/org/apache/storm/ui/core.clj | 99 +-
.../src/clj/org/apache/storm/ui/helpers.clj | 14 +-
storm-core/src/clj/org/apache/storm/util.clj | 925 +----------------
storm-core/src/jvm/org/apache/storm/Config.java | 39 +
.../storm/cluster/PaceMakerStateStorage.java | 4 +-
.../org/apache/storm/command/ConfigValue.java | 30 +
.../storm/daemon/metrics/MetricsUtils.java | 108 ++
.../reporters/ConsolePreparableReporter.java | 76 ++
.../reporters/CsvPreparableReporter.java | 80 ++
.../reporters/JmxPreparableReporter.java | 70 ++
.../metrics/reporters/PreparableReporter.java | 32 +
.../storm/logging/ThriftAccessLogger.java | 13 +-
.../apache/storm/pacemaker/PacemakerClient.java | 5 +
.../serialization/SerializationFactory.java | 17 +-
.../staticmocking/MockedConfigUtils.java | 31 -
.../jvm/org/apache/storm/trident/Stream.java | 87 +-
.../storm/trident/operation/Consumer.java | 35 +
.../trident/operation/FlatMapFunction.java | 37 +
.../storm/trident/operation/MapFunction.java | 36 +
.../operation/impl/ConsumerExecutor.java | 38 +
.../operation/impl/FlatMapFunctionExecutor.java | 43 +
.../operation/impl/MapFunctionExecutor.java | 41 +
.../trident/planner/processor/MapProcessor.java | 87 ++
.../jvm/org/apache/storm/utils/ConfigUtils.java | 20 +-
.../jvm/org/apache/storm/utils/Container.java | 11 +-
.../jvm/org/apache/storm/utils/IPredicate.java | 27 +
.../org/apache/storm/utils/NimbusClient.java | 2 +-
.../utils/StormConnectionStateConverter.java | 44 +
.../jvm/org/apache/storm/utils/TestUtils.java | 34 -
.../src/jvm/org/apache/storm/utils/Time.java | 26 +-
.../src/jvm/org/apache/storm/utils/Utils.java | 989 +++++++++++++++++--
.../storm/validation/ConfigValidation.java | 2 +-
.../org/apache/storm/zookeeper/Zookeeper.java | 7 +
.../org/apache/storm/integration_test.clj | 98 +-
.../org/apache/storm/testing4j_test.clj | 37 +-
.../apache/storm/trident/integration_test.clj | 15 +-
.../test/clj/org/apache/storm/cluster_test.clj | 27 +-
.../test/clj/org/apache/storm/drpc_test.clj | 23 +-
.../clj/org/apache/storm/logviewer_test.clj | 267 ++---
.../storm/messaging/netty_integration_test.clj | 2 +-
.../test/clj/org/apache/storm/nimbus_test.clj | 112 ++-
.../storm/pacemaker_state_factory_test.clj | 74 +-
.../scheduler/resource_aware_scheduler_test.clj | 21 +-
.../apache/storm/security/auth/auth_test.clj | 11 +-
.../authorizer/DRPCSimpleACLAuthorizer_test.clj | 2 +-
.../BlowfishTupleSerializer_test.clj | 1 -
.../clj/org/apache/storm/serialization_test.clj | 23 +-
.../clj/org/apache/storm/supervisor_test.clj | 649 ++++++------
.../clj/org/apache/storm/transactional_test.clj | 18 +
.../clj/org/apache/storm/trident/state_test.clj | 5 +-
.../clj/org/apache/storm/trident/tuple_test.clj | 15 +-
.../test/clj/org/apache/storm/utils_test.clj | 16 +-
.../test/clj/org/apache/storm/worker_test.clj | 1 -
.../staticmocking/ConfigUtilsInstaller.java | 38 +
.../utils/staticmocking/UtilsInstaller.java | 38 +
.../storm/utils/staticmocking/package-info.java | 95 ++
110 files changed, 4199 insertions(+), 2614 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/conf/defaults.yaml
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
index 7be526d,657e242..ab5cbed
--- a/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
+++ b/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
@@@ -14,7 -14,8 +14,8 @@@
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns org.apache.storm.command.dev-zookeeper
+ (:import [org.apache.storm.utils Utils])
- (:use [org.apache.storm zookeeper util config])
+ (:use [org.apache.storm util config])
(:import [org.apache.storm.utils ConfigUtils])
(:import [org.apache.storm.zookeeper Zookeeper])
(:gen-class))
http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/command/shell_submission.clj
index 3978d2f,0253338..870e7f6
--- a/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
+++ b/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
@@@ -15,8 -15,9 +15,9 @@@
;; limitations under the License.
(ns org.apache.storm.command.shell-submission
(:import [org.apache.storm StormSubmitter]
+ [org.apache.storm.utils Utils]
[org.apache.storm.zookeeper Zookeeper])
- (:use [org.apache.storm thrift util config log zookeeper])
+ (:use [org.apache.storm thrift util config log])
(:require [clojure.string :as str])
(:import [org.apache.storm.utils ConfigUtils])
(:gen-class))
http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/converter.clj
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/daemon/common.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/common.clj
index b144f40,eb1ec1e..db342d2
--- a/storm-core/src/clj/org/apache/storm/daemon/common.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/common.clj
@@@ -15,17 -15,19 +15,20 @@@
;; limitations under the License.
(ns org.apache.storm.daemon.common
(:use [org.apache.storm log config util])
- (:import [org.apache.storm.generated StormTopology
+ (:import [org.apache.storm.generated StormTopology NodeInfo
InvalidTopologyException GlobalStreamId]
- [org.apache.storm.utils ThriftTopologyUtils])
- (:import [org.apache.storm.utils Utils ConfigUtils])
+ [org.apache.storm.utils Utils ConfigUtils IPredicate ThriftTopologyUtils]
+ [org.apache.storm.daemon.metrics.reporters PreparableReporter]
+ [com.codahale.metrics MetricRegistry])
+ (:import [org.apache.storm.daemon.metrics MetricsUtils])
(:import [org.apache.storm.task WorkerTopologyContext])
(:import [org.apache.storm Constants])
+ (:import [org.apache.storm.cluster StormClusterStateImpl])
(:import [org.apache.storm.metric SystemBolt])
(:import [org.apache.storm.metric EventLoggerBolt])
- (:import [org.apache.storm.security.auth IAuthorizer])
- (:import [java.io InterruptedIOException])
+ (:import [org.apache.storm.security.auth IAuthorizer])
+ (:import [java.io InterruptedIOException]
+ [org.json.simple JSONValue])
(:require [clojure.set :as set])
(:require [org.apache.storm.daemon.acker :as acker])
(:require [org.apache.storm.thrift :as thrift])
@@@ -73,12 -83,10 +84,11 @@@
(defn new-executor-stats []
(ExecutorStats. 0 0 0 0 0))
+
(defn get-storm-id [storm-cluster-state storm-name]
- (let [active-storms (.activeStorms storm-cluster-state)]
- (find-first
- #(= storm-name (.get_name (.stormBase storm-cluster-state % nil)))
- active-storms)
- (let [active-storms (.active-storms storm-cluster-state)
- pred (reify IPredicate (test [this x] (= storm-name (:storm-name (.storm-base storm-cluster-state x nil)))))]
++ (let [active-storms (.activeStorms storm-cluster-state)
++ pred (reify IPredicate (test [this x] (= storm-name (.get_name (.stormBase storm-cluster-state x nil)))))]
+ (Utils/findOne pred active-storms)
))
(defn topology-bases [storm-cluster-state]
http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/executor.clj
index 49ae6cf,e2380b7..33b89ed
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@@ -34,10 -34,13 +34,12 @@@
(:import [org.apache.storm.daemon Shutdownable])
(:import [org.apache.storm.metric.api IMetric IMetricsConsumer$TaskInfo IMetricsConsumer$DataPoint StateMetric])
(:import [org.apache.storm Config Constants])
- (:import [org.apache.storm.cluster ClusterStateContext DaemonType])
+ (:import [org.apache.storm.cluster ClusterStateContext DaemonType StormClusterStateImpl ClusterUtils])
(:import [org.apache.storm.grouping LoadAwareCustomStreamGrouping LoadAwareShuffleGrouping LoadMapping ShuffleGrouping])
- (:import [java.util.concurrent ConcurrentLinkedQueue])
+ (:import [java.lang Thread Thread$UncaughtExceptionHandler]
+ [java.util.concurrent ConcurrentLinkedQueue]
+ [org.json.simple JSONValue])
- (:require [org.apache.storm [thrift :as thrift]
- [cluster :as cluster] [disruptor :as disruptor] [stats :as stats]])
+ (:require [org.apache.storm [thrift :as thrift] [disruptor :as disruptor] [stats :as stats]])
(:require [org.apache.storm.daemon [task :as task]])
(:require [org.apache.storm.daemon.builtin-metrics :as builtin-metrics])
(:require [clojure.set :as set]))
@@@ -206,9 -211,9 +210,9 @@@
(swap! interval-errors inc)
(when (<= @interval-errors max-per-interval)
- (cluster/report-error (:storm-cluster-state executor) (:storm-id executor) (:component-id executor)
+ (.reportError (:storm-cluster-state executor) (:storm-id executor) (:component-id executor)
- (hostname storm-conf)
+ (Utils/hostname storm-conf)
- (.getThisWorkerPort (:worker-context executor)) error)
+ (long (.getThisWorkerPort (:worker-context executor))) error)
))))
;; in its own function so that it can be mocked out by tracked topologies
http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index daf5e45,710cd83..6af1b81
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@@ -38,9 -39,9 +39,9 @@@
(:import [org.apache.storm.scheduler INimbus SupervisorDetails WorkerSlot TopologyDetails
Cluster Topologies SchedulerAssignment SchedulerAssignmentImpl DefaultScheduler ExecutorDetails])
(:import [org.apache.storm.nimbus NimbusInfo])
- (:import [org.apache.storm.utils TimeCacheMap TimeCacheMap$ExpiredCallback Utils ConfigUtils TupleUtils ThriftTopologyUtils
+ (:import [org.apache.storm.utils TimeCacheMap Time TimeCacheMap$ExpiredCallback Utils ConfigUtils TupleUtils ThriftTopologyUtils
BufferFileInputStream BufferInputStream])
- (:import [org.apache.storm.generated NotAliveException AlreadyAliveException StormTopology ErrorInfo
+ (:import [org.apache.storm.generated NotAliveException AlreadyAliveException StormTopology ErrorInfo ClusterWorkerHeartbeat
ExecutorInfo InvalidTopologyException Nimbus$Iface Nimbus$Processor SubmitOptions TopologyInitialStatus
KillOptions RebalanceOptions ClusterSummary SupervisorSummary TopologySummary TopologyInfo TopologyHistoryInfo
ExecutorSummary AuthorizationException GetInfoOptions NumErrorsChoice SettableBlobMeta ReadableBlobMeta
@@@ -407,10 -414,10 +409,10 @@@
[storm-cluster-state]
(let [assignments (.assignments storm-cluster-state nil)]
- (defaulted
+ (or
(apply merge-with set/union
(for [a assignments
- [_ [node port]] (-> (.assignment-info storm-cluster-state a nil) :executor->node+port)]
+ [_ [node port]] (-> (clojurify-assignment (.assignmentInfo storm-cluster-state a nil)) :executor->node+port)]
{node #{port}}
))
{})
@@@ -988,10 -1002,10 +1002,10 @@@
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
+ (.activateStorm storm-cluster-state
storm-id
- (StormBase. storm-name
+ (thriftify-storm-base (StormBase. storm-name
- (current-time-secs)
+ (Time/currentTimeSecs)
{:type topology-initial-status}
(storm-conf TOPOLOGY-WORKERS)
num-executors
@@@ -1137,9 -1152,9 +1152,9 @@@
(when-not (empty? to-cleanup-ids)
(doseq [id to-cleanup-ids]
(log-message "Cleaning up " id)
- (.teardown-heartbeats! storm-cluster-state id)
- (.teardown-topology-errors! storm-cluster-state id)
+ (.teardownHeartbeats storm-cluster-state id)
+ (.teardownTopologyErrors storm-cluster-state id)
- (rmr (ConfigUtils/masterStormDistRoot conf id))
+ (Utils/forceDelete (ConfigUtils/masterStormDistRoot conf id))
(blob-rm-topology-keys id blob-store storm-cluster-state)
(swap! (:heartbeats-cache nimbus) dissoc id)))))
(log-message "not a leader, skipping cleanup")))
@@@ -1811,8 -1830,8 +1838,8 @@@
(.set_used_cpu sup-sum used-cpu))
(when-let [version (:version info)] (.set_version sup-sum version))
sup-sum))
- nimbus-uptime ((:uptime nimbus))
+ nimbus-uptime (. (:uptime nimbus) upTime)
- bases (topology-bases storm-cluster-state)
+ bases (nimbus-topology-bases storm-cluster-state)
nimbuses (.nimbuses storm-cluster-state)
;;update the isLeader field for each nimbus summary
http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
index 3a83d03,ae9e92f..c1f058f
--- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
@@@ -16,14 -16,15 +16,15 @@@
(ns org.apache.storm.daemon.supervisor
(:import [java.io File IOException FileOutputStream])
(:import [org.apache.storm.scheduler ISupervisor]
- [org.apache.storm.utils LocalState Time Utils ConfigUtils]
+ [org.apache.storm.utils LocalState Time Utils Utils$ExitCodeCallable
+ ConfigUtils]
[org.apache.storm.daemon Shutdownable]
[org.apache.storm Constants]
- [org.apache.storm.cluster ClusterStateContext DaemonType]
+ [org.apache.storm.cluster ClusterStateContext DaemonType StormClusterStateImpl ClusterUtils]
[java.net JarURLConnection]
- [java.net URI]
+ [java.net URI URLDecoder]
[org.apache.commons.io FileUtils])
- (:use [org.apache.storm config util log timer local-state])
+ (:use [org.apache.storm config util log timer local-state converter])
(:import [org.apache.storm.generated AuthorizationException KeyNotFoundException WorkerResources])
(:import [org.apache.storm.utils NimbusLeaderNotFoundException VersionInfo])
(:import [java.nio.file Files StandardCopyOption])
@@@ -315,12 -319,14 +321,12 @@@
:shared-context shared-context
:isupervisor isupervisor
:active (atom true)
- :uptime (uptime-computer)
+ :uptime (Utils/makeUptimeComputer)
:version STORM-VERSION
:worker-thread-pids-atom (atom {})
- :storm-cluster-state (cluster/mk-storm-cluster-state conf :acls (when
- (Utils/isZkAuthenticationConfiguredStormServer
- conf)
- SUPERVISOR-ZK-ACLS)
- :context (ClusterStateContext. DaemonType/SUPERVISOR))
+ :storm-cluster-state (ClusterUtils/mkStormClusterState conf (when (Utils/isZkAuthenticationConfiguredStormServer conf)
+ SUPERVISOR-ZK-ACLS)
+ (ClusterStateContext. DaemonType/SUPERVISOR))
:local-state (ConfigUtils/supervisorState conf)
:supervisor-id (.getSupervisorId isupervisor)
:assignment-id (.getAssignmentId isupervisor)
@@@ -777,19 -791,19 +792,19 @@@
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!
+ heartbeat-fn (fn [] (.supervisorHeartbeat
(:storm-cluster-state supervisor)
(:supervisor-id supervisor)
- (thriftify-supervisor-info (->SupervisorInfo (current-time-secs)
- (->SupervisorInfo (Time/currentTimeSecs)
++ (thriftify-supervisor-info (->SupervisorInfo (Time/currentTimeSecs)
(:my-hostname supervisor)
(:assignment-id supervisor)
(keys @(:curr-assignment supervisor))
;; used ports
(.getMetadata isupervisor)
(conf SUPERVISOR-SCHEDULER-META)
- ((:uptime supervisor))
+ (. (:uptime supervisor) upTime)
(:version supervisor)
- (mk-supervisor-capacities conf))))]
+ (mk-supervisor-capacities conf)))))]
(heartbeat-fn)
;; should synchronize supervisor so it doesn't launch anything after being down (optimization)
@@@ -1162,7 -1183,7 +1184,8 @@@
(.readBlobTo blob-store (ConfigUtils/masterStormConfKey storm-id) (FileOutputStream. (ConfigUtils/supervisorStormConfPath tmproot)) nil)
(finally
(.shutdown blob-store)))
-- (FileUtils/moveDirectory (File. tmproot) (File. stormroot))
++ (try (FileUtils/moveDirectory (File. tmproot) (File. stormroot)) (catch Exception e))
++
(setup-storm-code-dir conf (clojurify-structure (ConfigUtils/readSupervisorStormConf conf storm-id)) stormroot)
(let [classloader (.getContextClassLoader (Thread/currentThread))
resources-jar (resources-jar)
http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/worker.clj
index ae5be57,fe8cfae..9863427
--- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
@@@ -19,16 -19,18 +19,18 @@@
(:require [clj-time.core :as time])
(:require [clj-time.coerce :as coerce])
(:require [org.apache.storm.daemon [executor :as executor]])
- (:require [org.apache.storm [disruptor :as disruptor] [cluster :as cluster]])
+ (:require [org.apache.storm [disruptor :as disruptor]])
(:require [clojure.set :as set])
- (:require [org.apache.storm.messaging.loader :as msg-loader])
(:import [java.util.concurrent Executors]
- [org.apache.storm.hooks IWorkerHook BaseWorkerHook])
- (:import [java.util ArrayList HashMap])
- (:import [org.apache.storm.utils Utils ConfigUtils TransferDrainer ThriftTopologyUtils WorkerBackpressureThread DisruptorQueue])
+ [org.apache.storm.hooks IWorkerHook BaseWorkerHook]
+ [uk.org.lidalia.sysoutslf4j.context SysOutOverSLF4J])
+ (:import [java.util ArrayList HashMap]
+ [java.util.concurrent.locks ReentrantReadWriteLock])
+ (:import [org.apache.commons.io FileUtils])
+ (:import [org.apache.storm.utils Utils ConfigUtils TransferDrainer ThriftTopologyUtils WorkerBackpressureThread DisruptorQueue Time])
(:import [org.apache.storm.grouping LoadMapping])
(:import [org.apache.storm.messaging TransportFactory])
- (:import [org.apache.storm.messaging TaskMessage IContext IConnection ConnectionWithStatus ConnectionWithStatus$Status])
+ (:import [org.apache.storm.messaging TaskMessage IContext IConnection ConnectionWithStatus ConnectionWithStatus$Status DeserializingConnectionCallback])
(:import [org.apache.storm.daemon Shutdownable])
(:import [org.apache.storm.serialization KryoTupleSerializer])
(:import [org.apache.storm.generated StormTopology])
http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
index a36da3a,be4361a..0000000
deleted file mode 100644,100644
--- a/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
+++ /dev/null
@@@ -1,122 -1,141 +1,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.
--
--(ns org.apache.storm.pacemaker.pacemaker-state-factory
-- (:require [org.apache.storm.pacemaker pacemaker]
- [org.apache.storm.cluster-state [zookeeper-state-factory :as zk-factory]]
-- [org.apache.storm
-- [config :refer :all]
- [cluster :refer :all]
-- [log :refer :all]
-- [util :as util]])
-- (:import [org.apache.storm.generated
-- HBExecutionException HBServerMessageType HBMessage
-- HBMessageData HBPulse]
- [org.apache.storm.cluster ZKStateStorage ClusterUtils IStateStorage]
- [org.apache.storm.cluster_state zookeeper_state_factory]
- [org.apache.storm.cluster ClusterState]
-- [org.apache.storm.pacemaker PacemakerClient])
-- (:gen-class
- :implements [org.apache.storm.cluster.StateStorageFactory]))
- :implements [org.apache.storm.cluster.ClusterStateFactory]))
--
--;; So we can mock the client for testing
--(defn makeClient [conf]
-- (PacemakerClient. conf))
--
--(defn makeZKState [conf auth-conf acls context]
- (ClusterUtils/mkStateStorage conf auth-conf acls context))
- (.mkState (zookeeper_state_factory.) conf auth-conf acls context))
--
--(def max-retries 10)
-
-(defn retry-on-exception
- "Retries specific function on exception based on retries count"
- [retries task-description f & args]
- (let [res (try {:value (apply f args)}
- (catch Exception e
- (if (<= 0 retries)
- (throw e)
- {:exception e})))]
- (if (:exception res)
- (do
- (log-error (:exception res) (str "Failed to " task-description ". Will make [" retries "] more attempts."))
- (recur (dec retries) task-description f args))
- (do
- (log-debug (str "Successful " task-description "."))
- (:value res)))))
--
--(defn -mkState [this conf auth-conf acls context]
-- (let [zk-state (makeZKState conf auth-conf acls context)
-- pacemaker-client (makeClient conf)]
--
-- (reify
- IStateStorage
- ClusterState
-- ;; Let these pass through to the zk-state. We only want to handle heartbeats.
-- (register [this callback] (.register zk-state callback))
-- (unregister [this callback] (.unregister zk-state callback))
-- (set_ephemeral_node [this path data acls] (.set_ephemeral_node zk-state path data acls))
-- (create_sequential [this path data acls] (.create_sequential zk-state path data acls))
-- (set_data [this path data acls] (.set_data zk-state path data acls))
-- (delete_node [this path] (.delete_node zk-state path))
-- (delete_node_blobstore [this path nimbus-host-port-info] (.delete_node_blobstore zk-state path nimbus-host-port-info))
-- (get_data [this path watch?] (.get_data zk-state path watch?))
-- (get_data_with_version [this path watch?] (.get_data_with_version zk-state path watch?))
-- (get_version [this path watch?] (.get_version zk-state path watch?))
-- (get_children [this path watch?] (.get_children zk-state path watch?))
-- (mkdirs [this path acls] (.mkdirs zk-state path acls))
-- (node_exists [this path watch?] (.node_exists zk-state path watch?))
-- (add_listener [this listener] (.add_listener zk-state listener))
-- (sync_path [this path] (.sync_path zk-state path))
--
-- (set_worker_hb [this path data acls]
- (util/retry-on-exception
- (retry-on-exception
-- max-retries
-- "set_worker_hb"
-- #(let [response
-- (.send pacemaker-client
-- (HBMessage. HBServerMessageType/SEND_PULSE
-- (HBMessageData/pulse
-- (doto (HBPulse.)
-- (.set_id path)
-- (.set_details data)))))]
-- (if (= (.get_type response) HBServerMessageType/SEND_PULSE_RESPONSE)
-- :ok
-- (throw (HBExecutionException. "Invalid Response Type"))))))
--
-- (delete_worker_hb [this path]
- (util/retry-on-exception
- (retry-on-exception
-- max-retries
-- "delete_worker_hb"
-- #(let [response
-- (.send pacemaker-client
-- (HBMessage. HBServerMessageType/DELETE_PATH
-- (HBMessageData/path path)))]
-- (if (= (.get_type response) HBServerMessageType/DELETE_PATH_RESPONSE)
-- :ok
-- (throw (HBExecutionException. "Invalid Response Type"))))))
--
-- (get_worker_hb [this path watch?]
- (util/retry-on-exception
- (retry-on-exception
-- max-retries
-- "get_worker_hb"
-- #(let [response
-- (.send pacemaker-client
-- (HBMessage. HBServerMessageType/GET_PULSE
-- (HBMessageData/path path)))]
-- (if (= (.get_type response) HBServerMessageType/GET_PULSE_RESPONSE)
-- (try
-- (.get_details (.get_pulse (.get_data response)))
-- (catch Exception e
-- (throw (HBExecutionException. (.toString e)))))
-- (throw (HBExecutionException. "Invalid Response Type"))))))
--
-- (get_worker_hb_children [this path watch?]
- (util/retry-on-exception
- (retry-on-exception
-- max-retries
-- "get_worker_hb_children"
-- #(let [response
-- (.send pacemaker-client
-- (HBMessage. HBServerMessageType/GET_ALL_NODES_FOR_PATH
-- (HBMessageData/path path)))]
-- (if (= (.get_type response) HBServerMessageType/GET_ALL_NODES_FOR_PATH_RESPONSE)
-- (try
-- (into [] (.get_pulseIds (.get_nodes (.get_data response))))
-- (catch Exception e
-- (throw (HBExecutionException. (.toString e)))))
-- (throw (HBExecutionException. "Invalid Response Type"))))))
--
-- (close [this]
-- (.close zk-state)
-- (.close pacemaker-client)))))
http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/stats.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/stats.clj
index 0bf1757,8b37fc3..8632ed3
--- a/storm-core/src/clj/org/apache/storm/stats.clj
+++ b/storm-core/src/clj/org/apache/storm/stats.clj
@@@ -24,8 -24,8 +24,9 @@@
ExecutorAggregateStats SpecificAggregateStats
SpoutAggregateStats TopologyPageInfo TopologyStats])
(:import [org.apache.storm.utils Utils])
+ (:import [org.apache.storm.cluster StormClusterStateImpl])
- (:import [org.apache.storm.metric.internal MultiCountStatAndMetric MultiLatencyStatAndMetric])
+ (:import [org.apache.storm.metric.internal MultiCountStatAndMetric MultiLatencyStatAndMetric]
+ [java.util Collection])
(:use [org.apache.storm log util])
(:use [clojure.math.numeric-tower :only [ceil]]))
http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/testing.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/testing.clj
index 5a0bdf2,c872742..eef7754
--- a/storm-core/src/clj/org/apache/storm/testing.clj
+++ b/storm-core/src/clj/org/apache/storm/testing.clj
@@@ -44,11 -45,12 +45,12 @@@
(:import [org.apache.storm.transactional.partitioned PartitionedTransactionalSpoutExecutor])
(:import [org.apache.storm.tuple Tuple])
(:import [org.apache.storm.generated StormTopology])
- (:import [org.apache.storm.task TopologyContext])
+ (:import [org.apache.storm.task TopologyContext]
+ (org.apache.storm.messaging IContext)
+ [org.json.simple JSONValue])
- (:require [org.apache.storm [zookeeper :as zk]])
+ (:import [org.apache.storm.cluster ZKStateStorage ClusterStateContext StormClusterStateImpl ClusterUtils])
- (:require [org.apache.storm.messaging.loader :as msg-loader])
(:require [org.apache.storm.daemon.acker :as acker])
- (:use [org.apache.storm cluster util thrift config log local-state]))
+ (:use [org.apache.storm util thrift config log local-state converter]))
(defn feeder-spout
[fields]
http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/thrift.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/thrift.clj
index 4dc21f9,779c1d1..7aab729
--- a/storm-core/src/clj/org/apache/storm/thrift.clj
+++ b/storm-core/src/clj/org/apache/storm/thrift.clj
@@@ -29,8 -29,9 +29,9 @@@
(:import [org.apache.storm.grouping CustomStreamGrouping])
(:import [org.apache.storm.topology TopologyBuilder])
(:import [org.apache.storm.clojure RichShellBolt RichShellSpout])
- (:import [org.apache.thrift.transport TTransport])
+ (:import [org.apache.thrift.transport TTransport]
+ (org.json.simple JSONValue))
- (:use [org.apache.storm util config log zookeeper]))
+ (:use [org.apache.storm util config log]))
(defn instantiate-java-object
[^JavaObject obj]
http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/ui/core.clj
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/util.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/util.clj
index 165d8ee,f685d12..72778bb
--- a/storm-core/src/clj/org/apache/storm/util.clj
+++ b/storm-core/src/clj/org/apache/storm/util.clj
@@@ -20,9 -20,8 +20,9 @@@
(:import [java.io FileReader FileNotFoundException])
(:import [java.nio.file Paths])
(:import [org.apache.storm Config])
- (:import [org.apache.storm.generated ErrorInfo])
- (:import [org.apache.storm.utils Time Container ClojureTimerTask Utils
- MutableObject MutableInt])
++ (:import [org.apache.storm.generated ErrorInfo])
+ (:import [org.apache.storm.utils Time ClojureTimerTask Utils
+ MutableObject])
(:import [org.apache.storm.security.auth NimbusPrincipal])
(:import [javax.security.auth Subject])
(:import [java.util UUID Random ArrayList List Collections])
@@@ -262,58 -163,9 +164,19 @@@
(instance? Boolean x) (boolean x)
true x))
s))
+; move this func form convert.clj due to cyclic load dependency
+(defn clojurify-error [^ErrorInfo error]
+ (if error
+ {
+ :error (.get_error error)
+ :time-secs (.get_error_time_secs error)
+ :host (.get_host error)
+ :port (.get_port error)
+ }
+ ))
- (defmacro with-file-lock
- [path & body]
- `(let [f# (File. ~path)
- _# (.createNewFile f#)
- rf# (RandomAccessFile. f# "rw")
- lock# (.. rf# (getChannel) (lock))]
- (try
- ~@body
- (finally
- (.release lock#)
- (.close rf#)))))
-
- (defn tokenize-path
- [^String path]
- (let [toks (.split path "/")]
- (vec (filter (complement empty?) toks))))
-
- (defn assoc-conj
- [m k v]
- (merge-with concat m {k [v]}))
-
- ;; returns [ones in first set not in second, ones in second set not in first]
- (defn set-delta
- [old curr]
- (let [s1 (set old)
- s2 (set curr)]
- [(set/difference s1 s2) (set/difference s2 s1)]))
-
- (defn parent-path
- [path]
- (let [toks (tokenize-path path)]
- (str "/" (str/join "/" (butlast toks)))))
-
- (defn toks->path
- [toks]
- (str "/" (str/join "/" toks)))
-
- (defn normalize-path
- [^String path]
- (toks->path (tokenize-path path)))
-
+ ;TODO: We're keeping this function around until all the code using it is properly tranlated to java
+ ;TODO: by properly having the for loop IN THE JAVA FUNCTION that originally used this function.
(defn map-val
[afn amap]
(into {}
http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
index 1226c55,0000000..a9c4d89
mode 100644,000000..100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
@@@ -1,212 -1,0 +1,212 @@@
+/**
+ * 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 org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.storm.callback.ZKStateChangedCallback;
+import org.apache.storm.generated.*;
+import org.apache.storm.pacemaker.PacemakerClient;
+import org.apache.storm.utils.Utils;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+public class PaceMakerStateStorage implements IStateStorage {
+
+ private static Logger LOG = LoggerFactory.getLogger(PaceMakerStateStorage.class);
+
+ private PacemakerClient pacemakerClient;
+ private IStateStorage stateStorage;
+ private static final int maxRetries = 10;
+
+ public PaceMakerStateStorage(PacemakerClient pacemakerClient, IStateStorage stateStorage) throws Exception {
+ this.pacemakerClient = pacemakerClient;
+ this.stateStorage = stateStorage;
+ }
+
+ @Override
+ public String register(ZKStateChangedCallback callback) {
+ return stateStorage.register(callback);
+ }
+
+ @Override
+ public void unregister(String id) {
+ stateStorage.unregister(id);
+ }
+
+ @Override
+ public String create_sequential(String path, byte[] data, List<ACL> acls) {
+ return stateStorage.create_sequential(path, data, acls);
+ }
+
+ @Override
+ public void mkdirs(String path, List<ACL> acls) {
+ stateStorage.mkdirs(path, acls);
+ }
+
+ @Override
+ public void delete_node(String path) {
+ stateStorage.delete_node(path);
+ }
+
+ @Override
+ public void set_ephemeral_node(String path, byte[] data, List<ACL> acls) {
+ stateStorage.set_ephemeral_node(path, data, acls);
+ }
+
+ @Override
+ public Integer get_version(String path, boolean watch) throws Exception {
+ return stateStorage.get_version(path, watch);
+ }
+
+ @Override
+ public boolean node_exists(String path, boolean watch) {
+ return stateStorage.node_exists(path, watch);
+ }
+
+ @Override
+ public List<String> get_children(String path, boolean watch) {
+ return stateStorage.get_children(path, watch);
+ }
+
+ @Override
+ public void close() {
+ stateStorage.close();
+ pacemakerClient.close();
+ }
+
+ @Override
+ public void set_data(String path, byte[] data, List<ACL> acls) {
+ stateStorage.set_data(path, data, acls);
+ }
+
+ @Override
+ public byte[] get_data(String path, boolean watch) {
+ return stateStorage.get_data(path, watch);
+ }
+
+ @Override
+ public APersistentMap get_data_with_version(String path, boolean watch) {
+ return stateStorage.get_data_with_version(path, watch);
+ }
+
+ @Override
+ public void set_worker_hb(String path, byte[] data, List<ACL> acls) {
+ int retry = maxRetries;
+ while (true) {
+ try {
+ HBPulse hbPulse = new HBPulse();
+ hbPulse.set_id(path);
+ hbPulse.set_details(data);
+ HBMessage message = new HBMessage(HBServerMessageType.SEND_PULSE, HBMessageData.pulse(hbPulse));
+ HBMessage response = pacemakerClient.send(message);
+ if (response.get_type() != HBServerMessageType.SEND_PULSE_RESPONSE) {
+ throw new HBExecutionException("Invalid Response Type");
+ }
+ LOG.debug("Successful set_worker_hb");
+ break;
+ } catch (Exception e) {
+ if (retry <= 0) {
+ throw Utils.wrapInRuntime(e);
+ }
+ LOG.error("{} Failed to set_worker_hb. Will make {} more attempts.", e.getMessage(), retry--);
+ }
+ }
+ }
+
+ @Override
+ public byte[] get_worker_hb(String path, boolean watch) {
+ int retry = maxRetries;
+ while (true) {
+ try {
+ HBMessage message = new HBMessage(HBServerMessageType.GET_PULSE, HBMessageData.path(path));
+ HBMessage response = pacemakerClient.send(message);
+ if (response.get_type() != HBServerMessageType.GET_PULSE_RESPONSE) {
+ throw new HBExecutionException("Invalid Response Type");
+ }
+ LOG.debug("Successful get_worker_hb");
+ return response.get_data().get_pulse().get_details();
+ } catch (Exception e) {
+ if (retry <= 0) {
+ throw Utils.wrapInRuntime(e);
+ }
+ LOG.error("{} Failed to get_worker_hb. Will make {} more attempts.", e.getMessage(), retry--);
+ }
+ }
+ }
+
+ @Override
+ public List<String> get_worker_hb_children(String path, boolean watch) {
+ int retry = maxRetries;
+ while (true) {
+ try {
- HBMessage message = new HBMessage(HBServerMessageType.GET_PULSE, HBMessageData.path(path));
++ HBMessage message = new HBMessage(HBServerMessageType.GET_ALL_NODES_FOR_PATH, HBMessageData.path(path));
+ HBMessage response = pacemakerClient.send(message);
+ if (response.get_type() != HBServerMessageType.GET_ALL_NODES_FOR_PATH_RESPONSE) {
+ throw new HBExecutionException("Invalid Response Type");
+ }
+ LOG.debug("Successful get_worker_hb");
+ return response.get_data().get_nodes().get_pulseIds();
+ } catch (Exception e) {
+ if (retry <= 0) {
+ throw Utils.wrapInRuntime(e);
+ }
+ LOG.error("{} Failed to get_worker_hb_children. Will make {} more attempts.", e.getMessage(), retry--);
+ }
+ }
+ }
+
+ @Override
+ public void delete_worker_hb(String path) {
+ int retry = maxRetries;
+ while (true) {
+ try {
- HBMessage message = new HBMessage(HBServerMessageType.GET_PULSE, HBMessageData.path(path));
++ HBMessage message = new HBMessage(HBServerMessageType.DELETE_PATH, HBMessageData.path(path));
+ HBMessage response = pacemakerClient.send(message);
+ if (response.get_type() != HBServerMessageType.DELETE_PATH_RESPONSE) {
+ throw new HBExecutionException("Invalid Response Type");
+ }
+ LOG.debug("Successful get_worker_hb");
+ break;
+ } catch (Exception e) {
+ if (retry <= 0) {
+ throw Utils.wrapInRuntime(e);
+ }
+ LOG.error("{} Failed to delete_worker_hb. Will make {} more attempts.", e.getMessage(), retry--);
+ }
+ }
+ }
+
+ @Override
+ public void add_listener(ConnectionStateListener listener) {
+ stateStorage.add_listener(listener);
+ }
+
+ @Override
+ public void sync_path(String path) {
+ stateStorage.sync_path(path);
+ }
+
+ @Override
+ public void delete_node_blobstore(String path, String nimbusHostPortInfo) {
+ stateStorage.delete_node_blobstore(path, nimbusHostPortInfo);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
index af0e8f3,34f3665..20d6deb
--- a/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
+++ b/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
@@@ -60,6 -60,6 +60,10 @@@ public class PacemakerClient implement
private StormBoundedExponentialBackoffRetry backoff = new StormBoundedExponentialBackoffRetry(100, 5000, 20);
private int retryTimes = 0;
++ //the constructor is invoked by pacemaker-state-factory-test
++ public PacemakerClient() {
++ bootstrap = new ClientBootstrap();
++ }
public PacemakerClient(Map config) {
String host = (String)config.get(Config.PACEMAKER_HOST);
@@@ -157,6 -157,7 +161,7 @@@
public String secretKey() {
return secret;
}
-
++ public HBMessage checkCaptured() {return null;}
public HBMessage send(HBMessage m) {
waitUntilReady();
LOG.debug("Sending message: {}", m.toString());
http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/test/clj/org/apache/storm/cluster_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/cluster_test.clj
index 39adb9e,b146cb0..6c32d54
--- a/storm-core/test/clj/org/apache/storm/cluster_test.clj
+++ b/storm-core/test/clj/org/apache/storm/cluster_test.clj
@@@ -22,11 -22,11 +22,11 @@@
(:import [org.mockito Mockito])
(:import [org.mockito.exceptions.base MockitoAssertionError])
(:import [org.apache.curator.framework CuratorFramework CuratorFrameworkFactory CuratorFrameworkFactory$Builder])
- (:import [org.apache.storm.utils Utils TestUtils ZookeeperAuthInfo ConfigUtils])
+ (:import [org.apache.storm.utils Time Utils ZookeeperAuthInfo ConfigUtils])
- (:import [org.apache.storm.cluster ClusterState])
+ (:import [org.apache.storm.cluster IStateStorage ZKStateStorage ClusterStateContext StormClusterStateImpl ClusterUtils])
(:import [org.apache.storm.zookeeper Zookeeper])
- (:import [org.apache.storm.testing.staticmocking MockedZookeeper])
- (:require [org.apache.storm [zookeeper :as zk]])
+ (:import [org.apache.storm.callback ZKStateChangedCallback])
+ (:import [org.apache.storm.testing.staticmocking MockedZookeeper MockedCluster])
(:require [conjure.core])
(:use [conjure core])
(:use [clojure test])
@@@ -39,14 -39,18 +39,18 @@@
(defn mk-state
([zk-port] (let [conf (mk-config zk-port)]
- (mk-distributed-cluster-state conf :auth-conf conf)))
+ (ClusterUtils/mkStateStorage conf conf nil (ClusterStateContext.))))
([zk-port cb]
- (let [ret (mk-state zk-port)]
- (.register ret cb)
- ret )))
+ (let [ret (mk-state zk-port)]
+ (.register ret cb)
+ ret)))
-(defn mk-storm-state [zk-port] (mk-storm-cluster-state (mk-config zk-port)))
+(defn mk-storm-state [zk-port] (ClusterUtils/mkStormClusterState (mk-config zk-port) nil (ClusterStateContext.)))
+ (defn barr
+ [& vals]
+ (byte-array (map byte vals)))
+
(deftest test-basics
(with-inprocess-zookeeper zk-port
(let [state (mk-state zk-port)]
@@@ -242,27 -244,26 +246,32 @@@
(is (.contains (:error error) target))
)))
++(defn- stringify-error [error]
++ (let [result (java.io.StringWriter.)
++ printer (java.io.PrintWriter. result)]
++ (.printStackTrace error printer)
++ (.toString result)))
(deftest test-storm-cluster-state-errors
(with-inprocess-zookeeper zk-port
(with-simulated-time
(let [state (mk-storm-state zk-port)]
- (.reportError state "a" "1" (local-hostname) 6700 (stringify-error (RuntimeException.)))
- (.report-error state "a" "1" (Utils/localHostname) 6700 (RuntimeException.))
++ (.reportError state "a" "1" (Utils/localHostname) 6700 (stringify-error (RuntimeException.)))
(validate-errors! state "a" "1" ["RuntimeException"])
(advance-time-secs! 1)
- (.reportError state "a" "1" (local-hostname) 6700 (stringify-error (IllegalArgumentException.)))
- (.report-error state "a" "1" (Utils/localHostname) 6700 (IllegalArgumentException.))
++ (.reportError state "a" "1" (Utils/localHostname) 6700 (stringify-error (IllegalArgumentException.)))
(validate-errors! state "a" "1" ["IllegalArgumentException" "RuntimeException"])
(doseq [i (range 10)]
- (.reportError state "a" "2" (local-hostname) 6700 (stringify-error (RuntimeException.)))
- (.report-error state "a" "2" (Utils/localHostname) 6700 (RuntimeException.))
++ (.reportError state "a" "2" (Utils/localHostname) 6700 (stringify-error (RuntimeException.)))
(advance-time-secs! 2))
(validate-errors! state "a" "2" (repeat 10 "RuntimeException"))
(doseq [i (range 5)]
- (.reportError state "a" "2" (local-hostname) 6700 (stringify-error (IllegalArgumentException.)))
- (.report-error state "a" "2" (Utils/localHostname) 6700 (IllegalArgumentException.))
++ (.reportError state "a" "2" (Utils/localHostname) 6700 (stringify-error (IllegalArgumentException.)))
(advance-time-secs! 2))
(validate-errors! state "a" "2" (concat (repeat 5 "IllegalArgumentException")
- (repeat 5 "RuntimeException")
- ))
+ (repeat 5 "RuntimeException")
+ ))
+
(.disconnect state)
))))
@@@ -300,12 -301,12 +309,12 @@@
(. (Mockito/when (.connectString builder (Mockito/anyString))) (thenReturn builder))
(. (Mockito/when (.connectionTimeoutMs builder (Mockito/anyInt))) (thenReturn builder))
(. (Mockito/when (.sessionTimeoutMs builder (Mockito/anyInt))) (thenReturn builder))
- (TestUtils/testSetupBuilder builder (str zk-port "/") conf (ZookeeperAuthInfo. conf))
+ (Utils/testSetupBuilder builder (str zk-port "/") conf (ZookeeperAuthInfo. conf))
(is (nil?
- (try
- (. (Mockito/verify builder) (authorization "digest" (.getBytes (conf STORM-ZOOKEEPER-AUTH-PAYLOAD))))
- (catch MockitoAssertionError e
- e)))))))
+ (try
+ (. (Mockito/verify builder) (authorization "digest" (.getBytes (conf STORM-ZOOKEEPER-AUTH-PAYLOAD))))
+ (catch MockitoAssertionError e
+ e)))))))
(deftest test-storm-state-callbacks
;; TODO finish
@@@ -313,17 -314,15 +322,17 @@@
(deftest test-cluster-state-default-acls
(testing "The default ACLs are empty."
- (let [zk-mock (Mockito/mock Zookeeper)]
+ (let [zk-mock (Mockito/mock Zookeeper)
+ curator-frameworke (reify CuratorFramework (^void close [this] nil))]
;; No need for when clauses because we just want to return nil
(with-open [_ (MockedZookeeper. zk-mock)]
- (stubbing [zk/mk-client (reify CuratorFramework (^void close [this] nil))]
- (mk-distributed-cluster-state {})
- (.mkdirsImpl (Mockito/verify zk-mock (Mockito/times 1)) (Mockito/any) (Mockito/anyString) (Mockito/eq nil)))))
- (stubbing [mk-distributed-cluster-state (reify ClusterState
- (register [this callback] nil)
- (mkdirs [this path acls] nil))]
- (mk-storm-cluster-state {})
- (verify-call-times-for mk-distributed-cluster-state 1)
- (verify-first-call-args-for-indices mk-distributed-cluster-state [4] nil))))
+ (. (Mockito/when (.mkClientImpl zk-mock (Mockito/anyMap) (Mockito/anyList) (Mockito/any) (Mockito/anyString) (Mockito/any) (Mockito/anyMap))) (thenReturn curator-frameworke))
+ (ClusterUtils/mkStateStorage {} nil nil (ClusterStateContext.))
+ (.mkdirsImpl (Mockito/verify zk-mock (Mockito/times 1)) (Mockito/any) (Mockito/anyString) (Mockito/eq nil))))
+ (let [distributed-state-storage (reify IStateStorage
+ (register [this callback] nil)
+ (mkdirs [this path acls] nil))
+ cluster-utils (Mockito/mock ClusterUtils)]
+ (with-open [mocked-cluster (MockedCluster. cluster-utils)]
- (. (Mockito/when (mkStateStorageImpl cluster-utils (Mockito/any) (Mockito/any) (Mockito/eq nil) (Mockito/any))) (thenReturn distributed-state-storage))
++ (. (Mockito/when (.mkStateStorageImpl cluster-utils (Mockito/any) (Mockito/any) (Mockito/eq nil) (Mockito/any))) (thenReturn distributed-state-storage))
+ (ClusterUtils/mkStormClusterState {} nil (ClusterStateContext.))))))
http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/test/clj/org/apache/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/nimbus_test.clj
index 772a232,70cb885..2a65efc
--- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj
+++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
@@@ -23,27 -23,33 +23,36 @@@
[org.apache.storm.nimbus InMemoryTopologyActionNotifier])
(:import [org.apache.storm.testing.staticmocking MockedZookeeper])
(:import [org.apache.storm.scheduler INimbus])
+ (:import [org.mockito Mockito])
+ (:import [org.mockito.exceptions.base MockitoAssertionError])
(:import [org.apache.storm.nimbus ILeaderElector NimbusInfo])
- (:import [org.apache.storm.testing.staticmocking MockedConfigUtils MockedCluster])
++ (:import [org.apache.storm.testing.staticmocking MockedCluster])
(:import [org.apache.storm.generated Credentials NotAliveException SubmitOptions
TopologyInitialStatus TopologyStatus AlreadyAliveException KillOptions RebalanceOptions
InvalidTopologyException AuthorizationException
LogConfig LogLevel LogLevelAction])
(:import [java.util HashMap])
(:import [java.io File])
- (:import [org.apache.storm.utils Time Utils ConfigUtils])
+ (:import [org.apache.storm.utils Time Utils Utils$UptimeComputer ConfigUtils IPredicate]
+ [org.apache.storm.utils.staticmocking ConfigUtilsInstaller UtilsInstaller])
(:import [org.apache.storm.zookeeper Zookeeper])
- (:import [org.apache.commons.io FileUtils])
+ (:import [org.apache.commons.io FileUtils]
+ [org.json.simple JSONValue])
- (:use [org.apache.storm testing MockAutoCred util config log timer zookeeper])
+ (:import [org.apache.storm.cluster StormClusterStateImpl ClusterStateContext ClusterUtils])
+ (:use [org.apache.storm testing MockAutoCred util config log timer converter])
(:use [org.apache.storm.daemon common])
(:require [conjure.core])
(:require [org.apache.storm
- [thrift :as thrift]
- [cluster :as cluster]])
+ [thrift :as thrift]])
(:use [conjure core]))
+ (defn- from-json
+ [^String str]
+ (if str
+ (clojurify-structure
+ (JSONValue/parse str))
+ nil))
+
(defn storm-component->task-info [cluster storm-name]
(let [storm-id (get-storm-id (:storm-cluster-state cluster) storm-name)
nimbus (:nimbus cluster)]
@@@ -72,8 -80,8 +83,8 @@@
(defn storm-num-workers [state storm-name]
(let [storm-id (get-storm-id state storm-name)
- assignment (.assignment-info state storm-id nil)]
+ assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
- (count (reverse-map (:executor->node+port assignment)))
+ (count (clojurify-structure (Utils/reverseMap (:executor->node+port assignment))))
))
(defn topology-nodes [state storm-name]
@@@ -95,9 -103,11 +106,11 @@@
set
)))
+ ;TODO: when translating this function, don't call map-val, but instead use an inline for loop.
+ ; map-val is a temporary kluge for clojure.
(defn topology-node-distribution [state storm-name]
(let [storm-id (get-storm-id state storm-name)
- assignment (.assignment-info state storm-id nil)]
+ assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
(->> assignment
:executor->node+port
vals
@@@ -124,19 -134,18 +137,18 @@@
(defn do-executor-heartbeat [cluster storm-id executor]
(let [state (:storm-cluster-state cluster)
- executor->node+port (:executor->node+port (.assignment-info state storm-id nil))
+ executor->node+port (:executor->node+port (clojurify-assignment (.assignmentInfo state storm-id nil)))
[node port] (get executor->node+port executor)
- curr-beat (.get-worker-heartbeat state storm-id node port)
+ curr-beat (clojurify-zk-worker-hb (.getWorkerHeartbeat state storm-id node port))
stats (:executor-stats curr-beat)]
- (.worker-heartbeat! state storm-id node port
- {:storm-id storm-id :time-secs (Time/currentTimeSecs) :uptime 10 :executor-stats (merge stats {executor (stats/render-stats! (stats/mk-bolt-stats 20))})}
+ (.workerHeartbeat state storm-id node port
- (thriftify-zk-worker-hb {:storm-id storm-id :time-secs (current-time-secs) :uptime 10 :executor-stats (merge stats {executor (stats/render-stats! (stats/mk-bolt-stats 20))})})
++ (thriftify-zk-worker-hb {:storm-id storm-id :time-secs (Time/currentTimeSecs) :uptime 10 :executor-stats (merge stats {executor (stats/render-stats! (stats/mk-bolt-stats 20))})})
)))
(defn slot-assignments [cluster storm-id]
(let [state (:storm-cluster-state cluster)
- assignment (.assignment-info state storm-id nil)]
+ assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
- (reverse-map (:executor->node+port assignment))
- ))
+ (clojurify-structure (Utils/reverseMap (:executor->node+port assignment)))))
(defn task-ids [cluster storm-id]
(let [nimbus (:nimbus cluster)]
@@@ -146,8 -155,10 +158,10 @@@
(defn topology-executors [cluster storm-id]
(let [state (:storm-cluster-state cluster)
- assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
- (keys (:executor->node+port assignment))
- assignment (.assignment-info state storm-id nil)
- ret-keys (keys (:executor->node+port assignment))
++ assignment (clojurify-assignment (.assignmentInfo state storm-id nil))
++ ret-keys (keys (:executor->node+port assignment))
+ _ (log-message "ret-keys: " (pr-str ret-keys)) ]
+ ret-keys
))
(defn check-distribution [items distribution]
@@@ -1350,23 -1399,27 +1402,29 @@@
NIMBUS-THRIFT-PORT 6666})
expected-acls nimbus/NIMBUS-ZK-ACLS
fake-inimbus (reify INimbus (getForcedScheduler [this] nil))
+ fake-cu (proxy [ConfigUtils] []
- (nimbusTopoHistoryStateImpl [conf] nil))
++ (nimbusTopoHistoryStateImpl [conf] nil))
+ fake-utils (proxy [Utils] []
+ (newInstanceImpl [_])
+ (makeUptimeComputer [] (proxy [Utils$UptimeComputer] []
- (upTime [] 0))))]
++ (upTime [] 0))))
+ cluster-utils (Mockito/mock ClusterUtils)]
- (with-open [_ (proxy [MockedConfigUtils] []
+ (with-open [_ (ConfigUtilsInstaller. fake-cu)
+ _ (UtilsInstaller. fake-utils)
++ _ (proxy [ConfigUtils] []
+ (nimbusTopoHistoryStateImpl [conf] nil))
zk-le (MockedZookeeper. (proxy [Zookeeper] []
- (zkLeaderElectorImpl [conf] nil)))]
+ (zkLeaderElectorImpl [conf] nil)))
+ mocked-cluster (MockedCluster. cluster-utils)]
(stubbing [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
- mk-timer nil
- nimbus/mk-scheduler nil]
- (nimbus/nimbus-data auth-conf fake-inimbus)
- (verify-call-times-for cluster/mk-storm-cluster-state 1)
- (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
- expected-acls))))))
+ 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-scheduler nil]
+ (nimbus/nimbus-data auth-conf fake-inimbus)
+ (.mkStormClusterStateImpl (Mockito/verify cluster-utils (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
+ )))))
(deftest test-file-bogus-download
(with-local-cluster [cluster :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0 TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
@@@ -1397,9 -1450,9 +1455,9 @@@
STORM-CLUSTER-MODE "local"
STORM-ZOOKEEPER-PORT zk-port
STORM-LOCAL-DIR nimbus-dir}))
- (bind cluster-state (cluster/mk-storm-cluster-state conf))
+ (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
(bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
- (sleep-secs 1)
+ (Time/sleepSecs 1)
(bind topology (thrift/mk-topology
{"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
{}))
@@@ -1429,10 -1482,10 +1487,10 @@@
STORM-ZOOKEEPER-PORT zk-port
STORM-LOCAL-DIR nimbus-dir
NIMBUS-TOPOLOGY-ACTION-NOTIFIER-PLUGIN (.getName InMemoryTopologyActionNotifier)}))
- (bind cluster-state (cluster/mk-storm-cluster-state conf))
+ (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
(bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
(bind notifier (InMemoryTopologyActionNotifier.))
- (sleep-secs 1)
+ (Time/sleepSecs 1)
(bind topology (thrift/mk-topology
{"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
{}))
http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
index 1a7bd2c,0925237..1c45266
--- a/storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
+++ b/storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
@@@ -19,10 -20,8 +19,11 @@@
(:import [org.apache.storm.generated
HBExecutionException HBNodes HBRecords
HBServerMessageType HBMessage HBMessageData HBPulse]
- [org.apache.storm.cluster ClusterStateContext PaceMakerStateStorageFactory]
- [org.apache.storm.cluster ClusterStateContext]
- [org.mockito Mockito Matchers]))
++ [org.apache.storm.cluster ClusterStateContext PaceMakerStateStorageFactory PaceMakerStateStorage]
+ [org.mockito Mockito Matchers])
+(:import [org.mockito.exceptions.base MockitoAssertionError])
++(:import [org.apache.storm.pacemaker PacemakerClient])
+(:import [org.apache.storm.testing.staticmocking MockedPaceMakerStateStorageFactory]))
(defn- string-to-bytes [string]
(byte-array (map int string)))
@@@ -30,26 -29,24 +31,23 @@@
(defn- bytes-to-string [bytez]
(apply str (map char bytez)))
--(defprotocol send-capture
-- (send [this something])
-- (check-captured [this]))
--
(defn- make-send-capture [response]
(let [captured (atom nil)]
-- (reify send-capture
-- (send [this something] (reset! captured something) response)
-- (check-captured [this] @captured))))
-
-(defmacro with-mock-pacemaker-client-and-state [client state response & body]
- `(let [~client (make-send-capture ~response)]
- (stubbing [psf/makeZKState nil
- psf/makeClient ~client]
- (let [~state (psf/-mkState nil nil nil nil (ClusterStateContext.))]
++ (proxy [PacemakerClient] []
++ (send [m] (reset! captured m) response)
++ (checkCaptured [] @captured))))
+
+(defmacro with-mock-pacemaker-client-and-state [client state pacefactory mock response & body]
+ `(let [~client (make-send-capture ~response)
+ ~pacefactory (Mockito/mock PaceMakerStateStorageFactory)]
+
+ (with-open [~mock (MockedPaceMakerStateStorageFactory. ~pacefactory)]
+ (. (Mockito/when (.initZKstateImpl ~pacefactory (Mockito/any) (Mockito/any) (Mockito/anyList) (Mockito/any))) (thenReturn nil))
+ (. (Mockito/when (.initMakeClientImpl ~pacefactory (Mockito/any))) (thenReturn ~client))
- (let [~state (.mkStore ~pacefactory nil nil nil (ClusterStateContext.))]
++ (let [~state (PaceMakerStateStorage. (PaceMakerStateStorageFactory/initMakeClient nil)
++ (PaceMakerStateStorageFactory/initZKstate nil nil nil nil))]
~@body))))
-
(deftest pacemaker_state_set_worker_hb
(testing "set_worker_hb"
(with-mock-pacemaker-client-and-state
@@@ -57,7 -54,7 +55,7 @@@
(HBMessage. HBServerMessageType/SEND_PULSE_RESPONSE nil)
(.set_worker_hb state "/foo" (string-to-bytes "data") nil)
-- (let [sent (.check-captured client)
++ (let [sent (.checkCaptured client)
pulse (.get_pulse (.get_data sent))]
(is (= (.get_type sent) HBServerMessageType/SEND_PULSE))
(is (= (.get_id pulse) "/foo"))
@@@ -65,13 -62,13 +63,12 @@@
(testing "set_worker_hb"
(with-mock-pacemaker-client-and-state
- client state
+ client state pacefactory mock
(HBMessage. HBServerMessageType/SEND_PULSE nil)
- (is (thrown? RuntimeException
- (is (thrown? HBExecutionException
-- (.set_worker_hb state "/foo" (string-to-bytes "data") nil))))))
++ (is (thrown? RuntimeException
++ (.set_worker_hb state "/foo" (string-to-bytes "data") nil))))))
--
(deftest pacemaker_state_delete_worker_hb
(testing "delete_worker_hb"
@@@ -80,74 -77,74 +77,75 @@@
(HBMessage. HBServerMessageType/DELETE_PATH_RESPONSE nil)
(.delete_worker_hb state "/foo/bar")
-- (let [sent (.check-captured client)]
++ (let [sent (.checkCaptured client)]
(is (= (.get_type sent) HBServerMessageType/DELETE_PATH))
(is (= (.get_path (.get_data sent)) "/foo/bar")))))
-- (testing "delete_worker_hb"
-- (with-mock-pacemaker-client-and-state
- client state pacefactory mock
- client state
-- (HBMessage. HBServerMessageType/DELETE_PATH nil)
--
- (is (thrown? RuntimeException
- (is (thrown? HBExecutionException
-- (.delete_worker_hb state "/foo/bar"))))))
++ (testing "delete_worker_hb"
++ (with-mock-pacemaker-client-and-state
++ client state pacefactory mock
++ (HBMessage. HBServerMessageType/DELETE_PATH nil)
++
++ (is (thrown? RuntimeException
++ (.delete_worker_hb state "/foo/bar"))))))
(deftest pacemaker_state_get_worker_hb
(testing "get_worker_hb"
(with-mock-pacemaker-client-and-state
- client state
+ client state pacefactory mock
(HBMessage. HBServerMessageType/GET_PULSE_RESPONSE
-- (HBMessageData/pulse
-- (doto (HBPulse.)
-- (.set_id "/foo")
-- (.set_details (string-to-bytes "some data")))))
++ (HBMessageData/pulse
++ (doto (HBPulse.)
++ (.set_id "/foo")
++ (.set_details (string-to-bytes "some data")))))
(.get_worker_hb state "/foo" false)
-- (let [sent (.check-captured client)]
++ (let [sent (.checkCaptured client)]
(is (= (.get_type sent) HBServerMessageType/GET_PULSE))
(is (= (.get_path (.get_data sent)) "/foo")))))
(testing "get_worker_hb - fail (bad response)"
(with-mock-pacemaker-client-and-state
- client state
+ client state pacefactory mock
(HBMessage. HBServerMessageType/GET_PULSE nil)
--
- (is (thrown? HBExecutionException
- (.get_worker_hb state "/foo" false)))))
-
++
+ (is (thrown? RuntimeException
- (.get_worker_hb state "/foo" false)))))
-
++ (.get_worker_hb state "/foo" false)))))
++
(testing "get_worker_hb - fail (bad data)"
(with-mock-pacemaker-client-and-state
- client state
+ client state pacefactory mock
(HBMessage. HBServerMessageType/GET_PULSE_RESPONSE nil)
--
- (is (thrown? HBExecutionException
- (.get_worker_hb state "/foo" false))))))
++
+ (is (thrown? RuntimeException
- (.get_worker_hb state "/foo" false))))))
++ (.get_worker_hb state "/foo" false))))))
(deftest pacemaker_state_get_worker_hb_children
(testing "get_worker_hb_children"
(with-mock-pacemaker-client-and-state
- client state
+ client state pacefactory mock
(HBMessage. HBServerMessageType/GET_ALL_NODES_FOR_PATH_RESPONSE
-- (HBMessageData/nodes
-- (HBNodes. [])))
++ (HBMessageData/nodes
++ (HBNodes. [])))
(.get_worker_hb_children state "/foo" false)
-- (let [sent (.check-captured client)]
++ (let [sent (.checkCaptured client)]
(is (= (.get_type sent) HBServerMessageType/GET_ALL_NODES_FOR_PATH))
(is (= (.get_path (.get_data sent)) "/foo")))))
(testing "get_worker_hb_children - fail (bad response)"
(with-mock-pacemaker-client-and-state
- client state
+ client state pacefactory mock
(HBMessage. HBServerMessageType/DELETE_PATH nil)
- (is (thrown? HBExecutionException
- (.get_worker_hb_children state "/foo" false)))))
+ (is (thrown? RuntimeException
- (.get_worker_hb_children state "/foo" false)))))
++ (.get_worker_hb_children state "/foo" false)))))
-- (testing "get_worker_hb_children - fail (bad data)"
++ (testing "get_worker_hb_children - fail (bad data)"
(with-mock-pacemaker-client-and-state
- client state
+ client state pacefactory mock
(HBMessage. HBServerMessageType/GET_ALL_NODES_FOR_PATH_RESPONSE nil)
- ;need been update due to HBExecutionException
-
- (is (thrown? HBExecutionException
- (.get_worker_hb_children state "/foo" false))))))
++
+ (is (thrown? RuntimeException
- (.get_worker_hb_children state "/foo" false))))))
++ (.get_worker_hb_children state "/foo" false))))))
++
[04/27] storm git commit: delete zookeeper.clj
zookeeper_state_factory.clj cluster.clj, but some tests still can't pass
Posted by bo...@apache.org.
delete zookeeper.clj zookeeper_state_factory.clj cluster.clj, but some tests still can't pass
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/9a79fb7d
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/9a79fb7d
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/9a79fb7d
Branch: refs/heads/master
Commit: 9a79fb7de0e824e73c294738521e892f1d81fbb0
Parents: 682d31c
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Wed Feb 3 20:28:05 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Wed Feb 3 20:28:05 2016 +0800
----------------------------------------------------------------------
conf/defaults.yaml | 2 +-
storm-core/src/clj/org/apache/storm/cluster.clj | 691 -------------------
.../cluster_state/zookeeper_state_factory.clj | 163 -----
.../org/apache/storm/command/dev_zookeeper.clj | 2 +-
.../clj/org/apache/storm/command/heartbeats.clj | 6 +-
.../apache/storm/command/shell_submission.clj | 2 +-
.../src/clj/org/apache/storm/converter.clj | 14 +-
.../src/clj/org/apache/storm/daemon/common.clj | 13 +-
.../clj/org/apache/storm/daemon/executor.clj | 12 +-
.../src/clj/org/apache/storm/daemon/nimbus.clj | 138 ++--
.../clj/org/apache/storm/daemon/supervisor.clj | 35 +-
.../src/clj/org/apache/storm/daemon/worker.clj | 43 +-
.../storm/pacemaker/pacemaker_state_factory.clj | 12 +-
storm-core/src/clj/org/apache/storm/stats.clj | 3 +-
storm-core/src/clj/org/apache/storm/testing.clj | 14 +-
storm-core/src/clj/org/apache/storm/thrift.clj | 2 +-
storm-core/src/clj/org/apache/storm/ui/core.clj | 2 +-
storm-core/src/clj/org/apache/storm/util.clj | 11 +
.../src/clj/org/apache/storm/zookeeper.clj | 75 --
.../jvm/org/apache/storm/callback/Callback.java | 3 +
.../jvm/org/apache/storm/cluster/Cluster.java | 38 +-
.../org/apache/storm/cluster/ClusterState.java | 2 +-
.../storm/cluster/DistributedClusterState.java | 7 +-
.../apache/storm/cluster/StormClusterState.java | 34 +-
.../storm/cluster/StormZkClusterState.java | 109 +--
.../testing/staticmocking/MockedCluster.java | 31 +
.../org/apache/storm/integration_test.clj | 15 +-
.../test/clj/org/apache/storm/cluster_test.clj | 103 ++-
.../test/clj/org/apache/storm/nimbus_test.clj | 148 ++--
.../storm/security/auth/nimbus_auth_test.clj | 3 +-
.../clj/org/apache/storm/supervisor_test.clj | 29 +-
.../test/jvm/org/apache/storm/ClusterTest.java | 22 +
32 files changed, 488 insertions(+), 1296 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 8873d12..74605bb 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -51,7 +51,7 @@ storm.auth.simple-white-list.users: []
storm.auth.simple-acl.users: []
storm.auth.simple-acl.users.commands: []
storm.auth.simple-acl.admins: []
-storm.cluster.state.store: "org.apache.storm.cluster_state.zookeeper_state_factory"
+storm.cluster.state.store: "org.apache.storm.cluster.StormZkClusterState"
storm.meta.serialization.delegate: "org.apache.storm.serialization.GzipThriftSerializationDelegate"
storm.codedistributor.class: "org.apache.storm.codedistributor.LocalFileSystemCodeDistributor"
storm.workers.artifacts.dir: "workers-artifacts"
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/cluster.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/cluster.clj b/storm-core/src/clj/org/apache/storm/cluster.clj
deleted file mode 100644
index 152423a..0000000
--- a/storm-core/src/clj/org/apache/storm/cluster.clj
+++ /dev/null
@@ -1,691 +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.
-
-(ns org.apache.storm.cluster
- (:import [org.apache.zookeeper.data Stat ACL Id]
- [org.apache.storm.generated SupervisorInfo Assignment StormBase ClusterWorkerHeartbeat ErrorInfo Credentials NimbusSummary
- LogConfig ProfileAction ProfileRequest NodeInfo]
- [java.io Serializable])
- (:import [org.apache.zookeeper KeeperException KeeperException$NoNodeException ZooDefs ZooDefs$Ids ZooDefs$Perms])
- (:import [org.apache.curator.framework CuratorFramework])
- (:import [org.apache.storm.utils Utils])
- (:import [org.apache.storm.cluster ClusterState ClusterStateContext ClusterStateListener ConnectionState])
- (:import [java.security MessageDigest])
- (:import [org.apache.zookeeper.server.auth DigestAuthenticationProvider])
- (:import [org.apache.storm.nimbus NimbusInfo])
- (:use [org.apache.storm util log config converter])
- (:require [org.apache.storm [zookeeper :as zk]])
- (:require [org.apache.storm.daemon [common :as common]]))
-
-(defn mk-topo-only-acls
- [topo-conf]
- (let [payload (.get topo-conf STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD)]
- (when (Utils/isZkAuthenticationConfiguredTopology topo-conf)
- [(first ZooDefs$Ids/CREATOR_ALL_ACL)
- (ACL. ZooDefs$Perms/READ (Id. "digest" (DigestAuthenticationProvider/generateDigest payload)))])))
-
-(defnk mk-distributed-cluster-state
- [conf :auth-conf nil :acls nil :context (ClusterStateContext.)]
- (let [clazz (Class/forName (or (conf STORM-CLUSTER-STATE-STORE)
- "org.apache.storm.cluster_state.zookeeper_state_factory"))
- state-instance (.newInstance clazz)]
- (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 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
- (add-nimbus-host! [this nimbus-id nimbus-summary])
-
- (active-storms [this])
- (storm-base [this storm-id callback])
- (get-worker-heartbeat [this storm-id node port])
- (get-worker-profile-requests [this storm-id nodeinfo thrift?])
- (get-topology-profile-requests [this storm-id thrift?])
- (set-worker-profile-request [this storm-id profile-request])
- (delete-topology-profile-requests [this storm-id profile-request])
- (executor-beats [this storm-id executor->node+port])
- (supervisors [this callback])
- (supervisor-info [this supervisor-id]) ;; returns nil if doesn't exist
- (setup-heartbeats! [this storm-id])
- (teardown-heartbeats! [this storm-id])
- (teardown-topology-errors! [this storm-id])
- (heartbeat-storms [this])
- (error-topologies [this])
- (set-topology-log-config! [this storm-id log-config])
- (topology-log-config [this storm-id cb])
- (worker-heartbeat! [this storm-id node port info])
- (remove-worker-heartbeat! [this storm-id node port])
- (supervisor-heartbeat! [this supervisor-id info])
- (worker-backpressure! [this storm-id node port info])
- (topology-backpressure [this storm-id callback])
- (setup-backpressure! [this storm-id])
- (remove-worker-backpressure! [this storm-id node port])
- (activate-storm! [this storm-id storm-base])
- (update-storm! [this storm-id new-elems])
- (remove-storm-base! [this storm-id])
- (set-assignment! [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])
- (set-credentials! [this storm-id creds topo-conf])
- (credentials [this storm-id callback])
- (disconnect [this]))
-
-(def ASSIGNMENTS-ROOT "assignments")
-(def CODE-ROOT "code")
-(def STORMS-ROOT "storms")
-(def SUPERVISORS-ROOT "supervisors")
-(def WORKERBEATS-ROOT "workerbeats")
-(def BACKPRESSURE-ROOT "backpressure")
-(def ERRORS-ROOT "errors")
-(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")
-(def PROFILERCONFIG-ROOT "profilerconfigs")
-
-(def ASSIGNMENTS-SUBTREE (str "/" ASSIGNMENTS-ROOT))
-(def STORMS-SUBTREE (str "/" STORMS-ROOT))
-(def SUPERVISORS-SUBTREE (str "/" SUPERVISORS-ROOT))
-(def WORKERBEATS-SUBTREE (str "/" WORKERBEATS-ROOT))
-(def BACKPRESSURE-SUBTREE (str "/" BACKPRESSURE-ROOT))
-(def ERRORS-SUBTREE (str "/" ERRORS-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))
-(def PROFILERCONFIG-SUBTREE (str "/" PROFILERCONFIG-ROOT))
-
-(defn supervisor-path
- [id]
- (str SUPERVISORS-SUBTREE "/" id))
-
-(defn assignment-path
- [id]
- (str ASSIGNMENTS-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]
- (str NIMBUSES-SUBTREE "/" id))
-
-(defn storm-path
- [id]
- (str STORMS-SUBTREE "/" id))
-
-(defn workerbeat-storm-root
- [storm-id]
- (str WORKERBEATS-SUBTREE "/" storm-id))
-
-(defn workerbeat-path
- [storm-id node port]
- (str (workerbeat-storm-root storm-id) "/" node "-" port))
-
-(defn backpressure-storm-root
- [storm-id]
- (str BACKPRESSURE-SUBTREE "/" storm-id))
-
-(defn backpressure-path
- [storm-id node port]
- (str (backpressure-storm-root storm-id) "/" node "-" port))
-
-(defn error-storm-root
- [storm-id]
- (str ERRORS-SUBTREE "/" storm-id))
-
-(defn error-path
- [storm-id component-id]
- (str (error-storm-root storm-id) "/" (url-encode component-id)))
-
-(def last-error-path-seg "last-error")
-
-(defn last-error-path
- [storm-id component-id]
- (str (error-storm-root storm-id)
- "/"
- (url-encode component-id)
- "-"
- last-error-path-seg))
-
-(defn credentials-path
- [storm-id]
- (str CREDENTIALS-SUBTREE "/" storm-id))
-
-(defn log-config-path
- [storm-id]
- (str LOGCONFIG-SUBTREE "/" storm-id))
-
-(defn profiler-config-path
- ([storm-id]
- (str PROFILERCONFIG-SUBTREE "/" storm-id))
- ([storm-id host port request-type]
- (str (profiler-config-path storm-id) "/" host "_" port "_" request-type)))
-
-(defn- issue-callback!
- [cb-atom]
- (let [cb @cb-atom]
- (reset! cb-atom nil)
- (when cb
- (cb))))
-
-(defn- issue-map-callback!
- [cb-atom id]
- (let [cb (@cb-atom id)]
- (swap! cb-atom dissoc id)
- (when cb
- (cb id))))
-
-(defn- maybe-deserialize
- [ser clazz]
- (when ser
- (Utils/deserialize ser clazz)))
-
-(defrecord TaskError [error time-secs host port])
-
-(defn- parse-error-path
- [^String p]
- (Long/parseLong (.substring p 1)))
-
-(defn convert-executor-beats
- "Ensures that we only return heartbeats for executors assigned to
- this worker."
- [executors worker-hb]
- (let [executor-stats (:executor-stats worker-hb)]
- (->> executors
- (map (fn [t]
- (if (contains? executor-stats t)
- {t {:time-secs (:time-secs worker-hb)
- :uptime (:uptime worker-hb)
- :stats (get executor-stats t)}})))
- (into {}))))
-
-;; Watches should be used for optimization. When ZK is reconnecting, they're not guaranteed to be called.
-(defnk mk-storm-cluster-state
- [cluster-state-spec :acls nil :context (ClusterStateContext.)]
- (let [[solo? cluster-state] (if (instance? ClusterState cluster-state-spec)
- [false cluster-state-spec]
- [true (mk-distributed-cluster-state cluster-state-spec :auth-conf cluster-state-spec :acls acls :context context)])
- assignment-info-callback (atom {})
- assignment-info-with-version-callback (atom {})
- assignment-version-callback (atom {})
- supervisors-callback (atom nil)
- 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 {})
- blobstore-callback (atom nil)
- credentials-callback (atom {})
- log-config-callback (atom {})
- state-id (.register
- cluster-state
- (fn [type path]
- (let [[subtree & args] (tokenize-path path)]
- (condp = subtree
- ASSIGNMENTS-ROOT (if (empty? args)
- (issue-callback! assignments-callback)
- (do
- (issue-map-callback! assignment-info-callback (first args))
- (issue-map-callback! assignment-version-callback (first args))
- (issue-map-callback! assignment-info-with-version-callback (first args))))
- SUPERVISORS-ROOT (issue-callback! supervisors-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 BLOBSTORE-SUBTREE NIMBUSES-SUBTREE
- LOGCONFIG-SUBTREE]]
- (.mkdirs cluster-state p acls))
- (reify
- StormClusterState
-
- (assignments
- [this callback]
- (when callback
- (reset! assignments-callback callback))
- (.get_children cluster-state ASSIGNMENTS-SUBTREE (not-nil? callback)))
-
- (assignment-info
- [this storm-id callback]
- (when callback
- (swap! assignment-info-callback assoc storm-id callback))
- (clojurify-assignment (maybe-deserialize (.get_data cluster-state (assignment-path storm-id) (not-nil? callback)) Assignment)))
-
- (assignment-info-with-version
- [this storm-id callback]
- (when callback
- (swap! assignment-info-with-version-callback assoc storm-id callback))
- (let [{data :data version :version}
- (.get_data_with_version cluster-state (assignment-path storm-id) (not-nil? callback))]
- {:data (clojurify-assignment (maybe-deserialize data Assignment))
- :version version}))
-
- (assignment-version
- [this storm-id callback]
- (when callback
- (swap! assignment-version-callback assoc storm-id callback))
- (.get_version cluster-state (assignment-path storm-id) (not-nil? callback)))
-
- ;; blobstore state
- (blobstore
- [this callback]
- (when callback
- (reset! blobstore-callback callback))
- (.sync_path cluster-state BLOBSTORE-SUBTREE)
- (.get_children cluster-state BLOBSTORE-SUBTREE (not-nil? callback)))
-
- (nimbuses
- [this]
- (map #(maybe-deserialize (.get_data cluster-state (nimbus-path %1) false) NimbusSummary)
- (.get_children cluster-state NIMBUSES-SUBTREE false)))
-
- (add-nimbus-host!
- [this nimbus-id nimbus-summary]
- ;explicit delete for ephmeral node to ensure this session creates the entry.
- (.delete_node cluster-state (nimbus-path nimbus-id))
-
- (.add_listener cluster-state (reify ClusterStateListener
- (^void stateChanged[this ^ConnectionState newState]
- (log-message "Connection state listener invoked, zookeeper connection state has changed to " newState)
- (if (.equals newState ConnectionState/RECONNECTED)
- (do
- (log-message "Connection state has changed to reconnected so setting nimbuses entry one more time")
- (.set_ephemeral_node cluster-state (nimbus-path nimbus-id) (Utils/serialize nimbus-summary) acls))))))
-
- (.set_ephemeral_node cluster-state (nimbus-path nimbus-id) (Utils/serialize nimbus-summary) acls))
-
- (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))
-
- (error-topologies
- [this]
- (.get_children cluster-state ERRORS-SUBTREE false))
-
- (get-worker-heartbeat
- [this storm-id node port]
- (let [worker-hb (.get_worker_hb cluster-state (workerbeat-path storm-id node port) false)]
- (if worker-hb
- (-> worker-hb
- (maybe-deserialize ClusterWorkerHeartbeat)
- clojurify-zk-worker-hb))))
-
- (executor-beats
- [this storm-id executor->node+port]
- ;; need to take executor->node+port in explicitly so that we don't run into a situation where a
- ;; long dead worker with a skewed clock overrides all the timestamps. By only checking heartbeats
- ;; with an assigned node+port, and only reading executors from that heartbeat that are actually assigned,
- ;; we avoid situations like that
- (let [node+port->executors (reverse-map executor->node+port)
- all-heartbeats (for [[[node port] executors] node+port->executors]
- (->> (get-worker-heartbeat this storm-id node port)
- (convert-executor-beats executors)
- ))]
- (apply merge all-heartbeats)))
-
- (supervisors
- [this callback]
- (when callback
- (reset! supervisors-callback callback))
- (.get_children cluster-state SUPERVISORS-SUBTREE (not-nil? callback)))
-
- (supervisor-info
- [this supervisor-id]
- (clojurify-supervisor-info (maybe-deserialize (.get_data cluster-state (supervisor-path supervisor-id) false) SupervisorInfo)))
-
- (topology-log-config
- [this storm-id cb]
- (when cb
- (swap! log-config-callback assoc storm-id cb))
- (maybe-deserialize (.get_data cluster-state (log-config-path storm-id) (not-nil? cb)) LogConfig))
-
- (set-topology-log-config!
- [this storm-id log-config]
- (.set_data cluster-state (log-config-path storm-id) (Utils/serialize log-config) acls))
-
- (set-worker-profile-request
- [this storm-id profile-request]
- (let [request-type (.get_action profile-request)
- host (.get_node (.get_nodeInfo profile-request))
- port (first (.get_port (.get_nodeInfo profile-request)))]
- (.set_data cluster-state
- (profiler-config-path storm-id host port request-type)
- (Utils/serialize profile-request)
- acls)))
-
- (get-topology-profile-requests
- [this storm-id thrift?]
- (let [path (profiler-config-path storm-id)
- requests (if (.node_exists cluster-state path false)
- (dofor [c (.get_children cluster-state path false)]
- (let [raw (.get_data cluster-state (str path "/" c) false)
- request (maybe-deserialize raw ProfileRequest)]
- (if thrift?
- request
- (clojurify-profile-request request)))))]
- requests))
-
- (delete-topology-profile-requests
- [this storm-id profile-request]
- (let [profile-request-inst (thriftify-profile-request profile-request)
- action (:action profile-request)
- host (:host profile-request)
- port (:port profile-request)]
- (.delete_node cluster-state
- (profiler-config-path storm-id host port action))))
-
- (get-worker-profile-requests
- [this storm-id node-info thrift?]
- (let [host (:host node-info)
- port (:port node-info)
- profile-requests (get-topology-profile-requests this storm-id thrift?)]
- (if thrift?
- (filter #(and (= host (.get_node (.get_nodeInfo %))) (= port (first (.get_port (.get_nodeInfo %)))))
- profile-requests)
- (filter #(and (= host (:host %)) (= port (:port %)))
- profile-requests))))
-
- (worker-heartbeat!
- [this storm-id node port info]
- (let [thrift-worker-hb (thriftify-zk-worker-hb info)]
- (if thrift-worker-hb
- (.set_worker_hb cluster-state (workerbeat-path storm-id node port) (Utils/serialize thrift-worker-hb) acls))))
-
- (remove-worker-heartbeat!
- [this storm-id node port]
- (.delete_worker_hb cluster-state (workerbeat-path storm-id node port)))
-
- (setup-heartbeats!
- [this storm-id]
- (.mkdirs cluster-state (workerbeat-storm-root storm-id) acls))
-
- (teardown-heartbeats!
- [this storm-id]
- (try-cause
- (.delete_worker_hb cluster-state (workerbeat-storm-root storm-id))
- (catch KeeperException e
- (log-warn-error e "Could not teardown heartbeats for " storm-id))))
-
- (worker-backpressure!
- [this storm-id node port on?]
- "if znode exists and to be not on?, delete; if exists and on?, do nothing;
- if not exists and to be on?, create; if not exists and not on?, do nothing"
- (let [path (backpressure-path storm-id node port)
- existed (.node_exists cluster-state path false)]
- (if existed
- (if (not on?)
- (.delete_node cluster-state path)) ;; delete the znode since the worker is not congested
- (if on?
- (.set_ephemeral_node cluster-state path nil acls))))) ;; create the znode since worker is congested
-
- (topology-backpressure
- [this storm-id callback]
- "if the backpresure/storm-id dir is empty, this topology has throttle-on, otherwise not."
- (when callback
- (swap! backpressure-callback assoc storm-id callback))
- (let [path (backpressure-storm-root storm-id)
- children (.get_children cluster-state path (not-nil? callback))]
- (> (count children) 0)))
-
- (setup-backpressure!
- [this storm-id]
- (.mkdirs cluster-state (backpressure-storm-root storm-id) acls))
-
- (remove-worker-backpressure!
- [this storm-id node port]
- (.delete_node cluster-state (backpressure-path storm-id node port)))
-
- (teardown-topology-errors!
- [this storm-id]
- (try-cause
- (.delete_node cluster-state (error-storm-root storm-id))
- (catch KeeperException e
- (log-warn-error e "Could not teardown errors for " storm-id))))
-
- (supervisor-heartbeat!
- [this supervisor-id info]
- (let [thrift-supervisor-info (thriftify-supervisor-info info)]
- (.set_ephemeral_node cluster-state (supervisor-path supervisor-id) (Utils/serialize thrift-supervisor-info) acls)))
-
- (activate-storm!
- [this storm-id storm-base]
- (let [thrift-storm-base (thriftify-storm-base storm-base)]
- (.set_data cluster-state (storm-path storm-id) (Utils/serialize thrift-storm-base) acls)))
-
- (update-storm!
- [this storm-id new-elems]
- (let [base (storm-base this storm-id nil)
- executors (:component->executors base)
- component->debug (:component->debug base)
- new-elems (update new-elems :component->executors (partial merge executors))
- new-elems (update new-elems :component->debug (partial merge-with merge component->debug))]
- (.set_data cluster-state (storm-path storm-id)
- (-> base
- (merge new-elems)
- thriftify-storm-base
- Utils/serialize)
- acls)))
-
- (storm-base
- [this storm-id callback]
- (when callback
- (swap! storm-base-callback assoc storm-id callback))
- (clojurify-storm-base (maybe-deserialize (.get_data cluster-state (storm-path storm-id) (not-nil? callback)) StormBase)))
-
- (remove-storm-base!
- [this storm-id]
- (.delete_node cluster-state (storm-path storm-id)))
-
- (set-assignment!
- [this storm-id info]
- (let [thrift-assignment (thriftify-assignment info)]
- (.set_data cluster-state (assignment-path storm-id) (Utils/serialize thrift-assignment) 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 (credentials-path storm-id))
- (.delete_node cluster-state (log-config-path storm-id))
- (.delete_node cluster-state (profiler-config-path storm-id))
- (remove-storm-base! this storm-id))
-
- (set-credentials!
- [this storm-id creds topo-conf]
- (let [topo-acls (mk-topo-only-acls topo-conf)
- path (credentials-path storm-id)
- thriftified-creds (thriftify-credentials creds)]
- (.set_data cluster-state path (Utils/serialize thriftified-creds) topo-acls)))
-
- (credentials
- [this storm-id callback]
- (when callback
- (swap! credentials-callback assoc storm-id callback))
- (clojurify-crdentials (maybe-deserialize (.get_data cluster-state (credentials-path storm-id) (not-nil? callback)) Credentials)))
-
- (report-error
- [this storm-id component-id node port error]
- (let [path (error-path storm-id component-id)
- last-error-path (last-error-path storm-id component-id)
- data (thriftify-error {:time-secs (current-time-secs) :error (stringify-error error) :host node :port port})
- _ (.mkdirs cluster-state path acls)
- ser-data (Utils/serialize data)
- _ (.mkdirs cluster-state path acls)
- _ (.create_sequential cluster-state (str path "/e") ser-data acls)
- _ (.set_data cluster-state last-error-path ser-data acls)
- to-kill (->> (.get_children cluster-state path false)
- (sort-by parse-error-path)
- reverse
- (drop 10))]
- (doseq [k to-kill]
- (.delete_node cluster-state (str path "/" k)))))
-
- (errors
- [this storm-id component-id]
- (let [path (error-path storm-id component-id)
- errors (if (.node_exists cluster-state path false)
- (dofor [c (.get_children cluster-state path false)]
- (if-let [data (-> (.get_data cluster-state
- (str path "/" c)
- false)
- (maybe-deserialize ErrorInfo)
- clojurify-error)]
- (map->TaskError data)))
- ())]
- (->> (filter not-nil? errors)
- (sort-by (comp - :time-secs)))))
-
- (last-error
- [this storm-id component-id]
- (let [path (last-error-path storm-id component-id)]
- (if (.node_exists cluster-state path false)
- (if-let [data (-> (.get_data cluster-state path false)
- (maybe-deserialize ErrorInfo)
- clojurify-error)]
- (map->TaskError data)))))
-
- (disconnect
- [this]
- (.unregister cluster-state state-id)
- (when solo?
- (.close cluster-state))))))
-
-;; daemons have a single thread that will respond to events
-;; start with initialize event
-;; callbacks add events to the thread's queue
-
-;; keeps in memory cache of the state, only for what client subscribes to. Any subscription is automatically kept in sync, and when there are changes, client is notified.
-;; master gives orders through state, and client records status in state (ephemerally)
-
-;; master tells nodes what workers to launch
-
-;; master writes this. supervisors and workers subscribe to this to understand complete topology. each storm is a map from nodes to workers to tasks to ports whenever topology changes everyone will be notified
-;; master includes timestamp of each assignment so that appropriate time can be given to each worker to start up
-;; /assignments/{storm id}
-
-;; which tasks they talk to, etc. (immutable until shutdown)
-;; everyone reads this in full to understand structure
-;; /tasks/{storm id}/{task id} ; just contains bolt id
-
-;; supervisors send heartbeats here, master doesn't subscribe but checks asynchronously
-;; /supervisors/status/{ephemeral node ids} ;; node metadata such as port ranges are kept here
-
-;; tasks send heartbeats here, master doesn't subscribe, just checks asynchronously
-;; /taskbeats/{storm id}/{ephemeral task id}
-
-;; contains data about whether it's started or not, tasks and workers subscribe to specific storm here to know when to shutdown
-;; master manipulates
-;; /storms/{storm id}
-
-;; Zookeeper flows:
-
-;; Master:
-;; job submit:
-;; 1. read which nodes are available
-;; 2. set up the worker/{storm}/{task} stuff (static)
-;; 3. set assignments
-;; 4. start storm - necessary in case master goes down, when goes back up can remember to take down the storm (2 states: on or off)
-
-;; Monitoring (or by checking when nodes go down or heartbeats aren't received):
-;; 1. read assignment
-;; 2. see which tasks/nodes are up
-;; 3. make new assignment to fix any problems
-;; 4. if a storm exists but is not taken down fully, ensure that storm takedown is launched (step by step remove tasks and finally remove assignments)
-
-;; masters only possible watches is on ephemeral nodes and tasks, and maybe not even
-
-;; Supervisor:
-;; 1. monitor /storms/* and assignments
-;; 2. local state about which workers are local
-;; 3. when storm is on, check that workers are running locally & start/kill if different than assignments
-;; 4. when storm is off, monitor tasks for workers - when they all die or don't hearbeat, kill the process and cleanup
-
-;; Worker:
-;; 1. On startup, start the tasks if the storm is on
-
-;; Task:
-;; 1. monitor assignments, reroute when assignments change
-;; 2. monitor storm (when storm turns off, error if assignments change) - take down tasks as master turns them off
-
-;; locally on supervisor: workers write pids locally on startup, supervisor deletes it on shutdown (associates pid with worker name)
-;; supervisor periodically checks to make sure processes are alive
-;; {rootdir}/workers/{storm id}/{worker id} ;; contains pid inside
-
-;; all tasks in a worker share the same cluster state
-;; workers, supervisors, and tasks subscribes to storm to know when it's started or stopped
-;; on stopped, master removes records in order (tasks need to subscribe to themselves to see if they disappear)
-;; when a master removes a worker, the supervisor should kill it (and escalate to kill -9)
-;; on shutdown, tasks subscribe to tasks that send data to them to wait for them to die. when node disappears, they can die
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/cluster_state/zookeeper_state_factory.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/cluster_state/zookeeper_state_factory.clj b/storm-core/src/clj/org/apache/storm/cluster_state/zookeeper_state_factory.clj
deleted file mode 100644
index dcfa8d8..0000000
--- a/storm-core/src/clj/org/apache/storm/cluster_state/zookeeper_state_factory.clj
+++ /dev/null
@@ -1,163 +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.
-
-(ns org.apache.storm.cluster-state.zookeeper-state-factory
- (:import [org.apache.curator.framework.state ConnectionStateListener]
- [org.apache.storm.zookeeper Zookeeper])
- (:import [org.apache.zookeeper KeeperException$NoNodeException CreateMode
- Watcher$Event$EventType Watcher$Event$KeeperState]
- [org.apache.storm.cluster ClusterState DaemonType])
- (:use [org.apache.storm cluster config log util])
- (:require [org.apache.storm [zookeeper :as zk]])
- (:gen-class
- :implements [org.apache.storm.cluster.ClusterStateFactory]))
-
-(defn -mkState [this conf auth-conf acls context]
- (let [zk (zk/mk-client conf (conf STORM-ZOOKEEPER-SERVERS) (conf STORM-ZOOKEEPER-PORT) :auth-conf auth-conf)]
- (Zookeeper/mkdirs zk (conf STORM-ZOOKEEPER-ROOT) acls)
- (.close zk))
- (let [callbacks (atom {})
- active (atom true)
- zk-writer (zk/mk-client conf
- (conf STORM-ZOOKEEPER-SERVERS)
- (conf STORM-ZOOKEEPER-PORT)
- :auth-conf auth-conf
- :root (conf STORM-ZOOKEEPER-ROOT)
- :watcher (fn [state type path]
- (when @active
- (when-not (= Watcher$Event$KeeperState/SyncConnected state)
- (log-warn "Received event " state ":" type ":" path " with disconnected Writer Zookeeper."))
- (when-not (= Watcher$Event$EventType/None type)
- (doseq [callback (vals @callbacks)]
- (callback type path))))))
- is-nimbus? (= (.getDaemonType context) DaemonType/NIMBUS)
- zk-reader (if is-nimbus?
- (zk/mk-client conf
- (conf STORM-ZOOKEEPER-SERVERS)
- (conf STORM-ZOOKEEPER-PORT)
- :auth-conf auth-conf
- :root (conf STORM-ZOOKEEPER-ROOT)
- :watcher (fn [state type path]
- (when @active
- (when-not (= Watcher$Event$KeeperState/SyncConnected state)
- (log-warn "Received event " state ":" type ":" path " with disconnected Reader Zookeeper."))
- (when-not (= Watcher$Event$EventType/None type)
- (doseq [callback (vals @callbacks)]
- (callback type path))))))
- zk-writer)]
- (reify
- ClusterState
-
- (register
- [this callback]
- (let [id (uuid)]
- (swap! callbacks assoc id callback)
- id))
-
- (unregister
- [this id]
- (swap! callbacks dissoc id))
-
- (set-ephemeral-node
- [this path data acls]
- (Zookeeper/mkdirs zk-writer (parent-path path) acls)
- (if (Zookeeper/exists zk-writer path false)
- (try-cause
- (Zookeeper/setData zk-writer path data) ; should verify that it's ephemeral
- (catch KeeperException$NoNodeException e
- (log-warn-error e "Ephemeral node disappeared between checking for existing and setting data")
- (Zookeeper/createNode zk-writer path data CreateMode/EPHEMERAL acls)))
- (Zookeeper/createNode zk-writer path data CreateMode/EPHEMERAL acls)))
-
- (create-sequential
- [this path data acls]
- (Zookeeper/createNode zk-writer path data CreateMode/PERSISTENT_SEQUENTIAL acls))
-
- (set-data
- [this path data acls]
- ;; note: this does not turn off any existing watches
- (if (Zookeeper/exists zk-writer path false)
- (Zookeeper/setData zk-writer path data)
- (do
- (Zookeeper/mkdirs zk-writer (parent-path path) acls)
- (Zookeeper/createNode zk-writer path data CreateMode/PERSISTENT acls))))
-
- (set-worker-hb
- [this path data acls]
- (.set_data this path data acls))
-
- (delete-node
- [this path]
- (Zookeeper/deleteNode zk-writer path))
-
- (delete-worker-hb
- [this path]
- (.delete_node this path))
-
- (get-data
- [this path watch?]
- (Zookeeper/getData zk-reader path watch?))
-
- (get-data-with-version
- [this path watch?]
- (Zookeeper/getDataWithVersion zk-reader path watch?))
-
- (get-version
- [this path watch?]
- (Zookeeper/getVersion zk-reader path watch?))
-
- (get-worker-hb
- [this path watch?]
- (.get_data this path watch?))
-
- (get-children
- [this path watch?]
- (Zookeeper/getChildren zk-reader path watch?))
-
- (get-worker-hb-children
- [this path watch?]
- (.get_children this path watch?))
-
- (mkdirs
- [this path acls]
- (Zookeeper/mkdirs zk-writer path acls))
-
- (node-exists
- [this path watch?]
- (Zookeeper/existsNode zk-reader path watch?))
-
- (add-listener
- [this listener]
- (let [curator-listener (reify ConnectionStateListener
- (stateChanged
- [this client newState]
- (.stateChanged listener client newState)))]
- (Zookeeper/addListener zk-reader curator-listener)))
-
- (sync-path
- [this path]
- (Zookeeper/syncPath zk-writer path))
-
- (delete-node-blobstore
- [this path nimbus-host-port-info]
- (Zookeeper/deleteNodeBlobstore zk-writer path nimbus-host-port-info))
-
- (close
- [this]
- (reset! active false)
- (.close zk-writer)
- (if is-nimbus?
- (.close zk-reader))))))
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj b/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
index ef9ecbb..7be526d 100644
--- a/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
+++ b/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
@@ -14,7 +14,7 @@
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns org.apache.storm.command.dev-zookeeper
- (:use [org.apache.storm zookeeper util config])
+ (:use [org.apache.storm util config])
(:import [org.apache.storm.utils ConfigUtils])
(:import [org.apache.storm.zookeeper Zookeeper])
(:gen-class))
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/command/heartbeats.clj b/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
index be8d030..954042f 100644
--- a/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
+++ b/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
@@ -18,16 +18,16 @@
[config :refer :all]
[log :refer :all]
[util :refer :all]
- [cluster :refer :all]
[converter :refer :all]]
[clojure.string :as string])
(:import [org.apache.storm.generated ClusterWorkerHeartbeat]
- [org.apache.storm.utils Utils ConfigUtils])
+ [org.apache.storm.utils Utils ConfigUtils]
+ [org.apache.storm.cluster DistributedClusterState ClusterStateContext])
(:gen-class))
(defn -main [command path & args]
(let [conf (clojurify-structure (ConfigUtils/readStormConfig))
- cluster (mk-distributed-cluster-state conf :auth-conf conf)]
+ cluster (DistributedClusterState. conf conf nil (ClusterStateContext.))]
(println "Command: [" command "]")
(condp = command
"list"
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/command/shell_submission.clj b/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
index 8a5eb21..3978d2f 100644
--- a/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
+++ b/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
@@ -16,7 +16,7 @@
(ns org.apache.storm.command.shell-submission
(:import [org.apache.storm StormSubmitter]
[org.apache.storm.zookeeper Zookeeper])
- (:use [org.apache.storm thrift util config log zookeeper])
+ (:use [org.apache.storm thrift util config log])
(:require [clojure.string :as str])
(:import [org.apache.storm.utils ConfigUtils])
(:gen-class))
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/converter.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/converter.clj b/storm-core/src/clj/org/apache/storm/converter.clj
index bb2dc87..d169301 100644
--- a/storm-core/src/clj/org/apache/storm/converter.clj
+++ b/storm-core/src/clj/org/apache/storm/converter.clj
@@ -181,9 +181,9 @@
(defn thriftify-storm-base [storm-base]
(doto (StormBase.)
(.set_name (:storm-name storm-base))
- (.set_launch_time_secs (int (:launch-time-secs storm-base)))
+ (.set_launch_time_secs (if (:launch-time-secs storm-base) (int (:launch-time-secs storm-base)) 0))
(.set_status (convert-to-status-from-symbol (:status storm-base)))
- (.set_num_workers (int (:num-workers storm-base)))
+ (.set_num_workers (if (:num-workers storm-base) (int (:num-workers storm-base)) 0))
(.set_component_executors (map-val int (:component->executors storm-base)))
(.set_owner (:owner storm-base))
(.set_topology_action_options (thriftify-topology-action-options storm-base))
@@ -234,16 +234,6 @@
(.set_executor_stats (thriftify-stats (filter second (:executor-stats worker-hb))))
(.set_time_secs (:time-secs worker-hb)))))
-(defn clojurify-error [^ErrorInfo error]
- (if error
- {
- :error (.get_error error)
- :time-secs (.get_error_time_secs error)
- :host (.get_host error)
- :port (.get_port error)
- }
- ))
-
(defn thriftify-error [error]
(doto (ErrorInfo. (:error error) (:time-secs error))
(.set_host (:host error))
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/daemon/common.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/common.clj b/storm-core/src/clj/org/apache/storm/daemon/common.clj
index 6c184fd..c9534f4 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/common.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/common.clj
@@ -13,14 +13,16 @@
;; 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.
+;TopologyActionOptions TopologyStatus StormBase RebalanceOptions KillOptions
(ns org.apache.storm.daemon.common
(:use [org.apache.storm log config util])
- (:import [org.apache.storm.generated StormTopology
+ (:import [org.apache.storm.generated StormTopology NodeInfo
InvalidTopologyException GlobalStreamId]
[org.apache.storm.utils ThriftTopologyUtils])
(:import [org.apache.storm.utils Utils ConfigUtils])
(:import [org.apache.storm.task WorkerTopologyContext])
(:import [org.apache.storm Constants])
+ (:import [org.apache.storm.cluster StormZkClusterState])
(:import [org.apache.storm.metric SystemBolt])
(:import [org.apache.storm.metric EventLoggerBolt])
(:import [org.apache.storm.security.auth IAuthorizer])
@@ -72,18 +74,19 @@
(defn new-executor-stats []
(ExecutorStats. 0 0 0 0 0))
+
(defn get-storm-id [storm-cluster-state storm-name]
- (let [active-storms (.active-storms storm-cluster-state)]
+ (let [active-storms (.activeStorms storm-cluster-state)]
(find-first
- #(= storm-name (:storm-name (.storm-base storm-cluster-state % nil)))
+ #(= storm-name (.get_name (.stormBase storm-cluster-state % nil)))
active-storms)
))
(defn topology-bases [storm-cluster-state]
- (let [active-topologies (.active-storms storm-cluster-state)]
+ (let [active-topologies (.activeStorms storm-cluster-state)]
(into {}
(dofor [id active-topologies]
- [id (.storm-base storm-cluster-state id nil)]
+ [id (.stormBase storm-cluster-state id nil)]
))
))
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/executor.clj b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
index 82d56a9..e50e150 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@ -34,11 +34,10 @@
(:import [org.apache.storm.daemon Shutdownable])
(:import [org.apache.storm.metric.api IMetric IMetricsConsumer$TaskInfo IMetricsConsumer$DataPoint StateMetric])
(:import [org.apache.storm Config Constants])
- (:import [org.apache.storm.cluster ClusterStateContext DaemonType])
+ (:import [org.apache.storm.cluster ClusterStateContext DaemonType StormZkClusterState Cluster])
(:import [org.apache.storm.grouping LoadAwareCustomStreamGrouping LoadAwareShuffleGrouping LoadMapping ShuffleGrouping])
(:import [java.util.concurrent ConcurrentLinkedQueue])
- (:require [org.apache.storm [thrift :as thrift]
- [cluster :as cluster] [disruptor :as disruptor] [stats :as stats]])
+ (:require [org.apache.storm [thrift :as thrift] [disruptor :as disruptor] [stats :as stats]])
(:require [org.apache.storm.daemon [task :as task]])
(:require [org.apache.storm.daemon.builtin-metrics :as builtin-metrics])
(:require [clojure.set :as set]))
@@ -207,7 +206,7 @@
(swap! interval-errors inc)
(when (<= @interval-errors max-per-interval)
- (cluster/report-error (:storm-cluster-state executor) (:storm-id executor) (:component-id executor)
+ (.reportError (:storm-cluster-state executor) (:storm-id executor) (:component-id executor)
(hostname storm-conf)
(.getThisWorkerPort (:worker-context executor)) error)
))))
@@ -252,9 +251,8 @@
:batch-transfer-queue batch-transfer->worker
:transfer-fn (mk-executor-transfer-fn batch-transfer->worker storm-conf)
:suicide-fn (:suicide-fn worker)
- :storm-cluster-state (cluster/mk-storm-cluster-state (:cluster-state worker)
- :acls (Utils/getWorkerACL storm-conf)
- :context (ClusterStateContext. DaemonType/WORKER))
+ :storm-cluster-state (StormZkClusterState. (:cluster-state worker) (Utils/getWorkerACL storm-conf)
+ (ClusterStateContext. DaemonType/WORKER))
:type executor-type
;; TODO: should refactor this to be part of the executor specific map (spout or bolt with :common field)
:stats (mk-executor-stats <> (ConfigUtils/samplingRate storm-conf))
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index de5a14e..9b00df3 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@ -40,7 +40,7 @@
(:import [org.apache.storm.nimbus NimbusInfo])
(:import [org.apache.storm.utils TimeCacheMap TimeCacheMap$ExpiredCallback Utils ConfigUtils TupleUtils ThriftTopologyUtils
BufferFileInputStream BufferInputStream])
- (:import [org.apache.storm.generated NotAliveException AlreadyAliveException StormTopology ErrorInfo
+ (:import [org.apache.storm.generated NotAliveException AlreadyAliveException StormTopology ErrorInfo ClusterWorkerHeartbeat
ExecutorInfo InvalidTopologyException Nimbus$Iface Nimbus$Processor SubmitOptions TopologyInitialStatus
KillOptions RebalanceOptions ClusterSummary SupervisorSummary TopologySummary TopologyInfo TopologyHistoryInfo
ExecutorSummary AuthorizationException GetInfoOptions NumErrorsChoice SettableBlobMeta ReadableBlobMeta
@@ -48,10 +48,9 @@
ProfileRequest ProfileAction NodeInfo])
(:import [org.apache.storm.daemon Shutdownable])
(:import [org.apache.storm.validation ConfigValidation])
- (:import [org.apache.storm.cluster ClusterStateContext DaemonType])
- (:use [org.apache.storm util config log timer zookeeper local-state])
- (:require [org.apache.storm [cluster :as cluster]
- [converter :as converter]
+ (:import [org.apache.storm.cluster ClusterStateContext DaemonType StormZkClusterState])
+ (:use [org.apache.storm util config log timer local-state converter])
+ (:require [org.apache.storm [converter :as converter]
[stats :as stats]])
(:require [clojure.set :as set])
(:import [org.apache.storm.daemon.common StormBase Assignment])
@@ -174,11 +173,11 @@
:authorization-handler (mk-authorization-handler (conf NIMBUS-AUTHORIZER) conf)
:impersonation-authorization-handler (mk-authorization-handler (conf NIMBUS-IMPERSONATION-AUTHORIZER) conf)
:submitted-count (atom 0)
- :storm-cluster-state (cluster/mk-storm-cluster-state conf :acls (when
+ :storm-cluster-state (StormZkClusterState. conf (when
(Utils/isZkAuthenticationConfiguredStormServer
conf)
NIMBUS-ZK-ACLS)
- :context (ClusterStateContext. DaemonType/NIMBUS))
+ (ClusterStateContext. DaemonType/NIMBUS))
:submit-lock (Object.)
:cred-update-lock (Object.)
:log-update-lock (Object.)
@@ -275,11 +274,11 @@
(defn do-rebalance [nimbus storm-id status storm-base]
(let [rebalance-options (:topology-action-options storm-base)]
- (.update-storm! (:storm-cluster-state nimbus)
+ (.updateStorm (:storm-cluster-state nimbus)
storm-id
- (-> {:topology-action-options nil}
+ (thriftify-storm-base (-> {:topology-action-options nil}
(assoc-non-nil :component->executors (:component->executors rebalance-options))
- (assoc-non-nil :num-workers (:num-workers rebalance-options)))))
+ (assoc-non-nil :num-workers (:num-workers rebalance-options))))))
(mk-assignments nimbus :scratch-topology-id storm-id))
(defn state-transitions [nimbus storm-id status storm-base]
@@ -303,12 +302,12 @@
:kill (kill-transition nimbus storm-id)
:remove (fn []
(log-message "Killing topology: " storm-id)
- (.remove-storm! (:storm-cluster-state nimbus)
+ (.removeStorm (: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)))
+ (.removeBlobstoreKey (:storm-cluster-state nimbus) blob-key)
+ (.removeKeyVersion (:storm-cluster-state nimbus) blob-key)))
nil)
}
:rebalancing {:startup (fn [] (delay-event nimbus
@@ -332,7 +331,7 @@
(locking (:submit-lock nimbus)
(let [system-events #{:startup}
[event & event-args] (if (keyword? event) [event] event)
- storm-base (-> nimbus :storm-cluster-state (.storm-base storm-id nil))
+ storm-base (clojurify-storm-base (-> nimbus :storm-cluster-state (.stormBase storm-id nil)))
status (:status storm-base)]
;; handles the case where event was scheduled but topology has been removed
(if-not status
@@ -362,7 +361,7 @@
storm-base-updates)]
(when storm-base-updates
- (.update-storm! (:storm-cluster-state nimbus) storm-id storm-base-updates)))))
+ (.updateStorm (:storm-cluster-state nimbus) storm-id (thriftify-storm-base storm-base-updates))))))
)))
(defn transition-name! [nimbus storm-name event & args]
@@ -411,7 +410,7 @@
(defaulted
(apply merge-with set/union
(for [a assignments
- [_ [node port]] (-> (.assignment-info storm-cluster-state a nil) :executor->node+port)]
+ [_ [node port]] (-> (clojurify-assignment (.assignmentInfo storm-cluster-state a nil)) :executor->node+port)]
{node #{port}}
))
{})
@@ -424,7 +423,7 @@
(into {}
(mapcat
(fn [id]
- (if-let [info (.supervisor-info storm-cluster-state id)]
+ (if-let [info (clojurify-supervisor-info (.supervisorInfo storm-cluster-state id))]
[[id info]]
))
supervisor-ids))
@@ -469,13 +468,13 @@
(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))))
+ (.setupBlobstore 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)))
+ (.setupBlobstore 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)))))
+ (.setupBlobstore 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
@@ -540,7 +539,7 @@
(defn read-topology-details [nimbus storm-id]
(let [blob-store (:blob-store nimbus)
storm-base (or
- (.storm-base (:storm-cluster-state nimbus) storm-id nil)
+ (clojurify-storm-base (.stormBase (:storm-cluster-state nimbus) storm-id nil))
(throw (NotAliveException. storm-id)))
topology-conf (read-storm-conf-as-nimbus storm-id blob-store)
topology (read-storm-topology-as-nimbus storm-id blob-store)
@@ -587,7 +586,12 @@
(defn update-heartbeats! [nimbus storm-id all-executors existing-assignment]
(log-debug "Updating heartbeats for " storm-id " " (pr-str all-executors))
(let [storm-cluster-state (:storm-cluster-state nimbus)
- executor-beats (.executor-beats storm-cluster-state storm-id (:executor->node+port existing-assignment))
+ executor-beats (let [executor-stats-java-map (.executorBeats storm-cluster-state storm-id (.get_executor_node_port (thriftify-assignment existing-assignment)))]
+ (->> (clojurify-structure executor-stats-java-map)
+ (map (fn [^ExecutorInfo executor-info ^ClusterWorkerHeartbeat cluster-worker-heartbeat]
+ {[(.get_task_start executor-info) (.get_task_end executor-info)] (clojurify-zk-worker-hb cluster-worker-heartbeat)}))
+ (into {})))
+
cache (update-heartbeat-cache (@(:heartbeats-cache nimbus) storm-id)
executor-beats
all-executors
@@ -637,7 +641,7 @@
(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)
+ storm-base (clojurify-storm-base (.stormBase (:storm-cluster-state nimbus) storm-id nil))
component->executors (:component->executors storm-base)
storm-conf (read-storm-conf-as-nimbus storm-id blob-store)
topology (read-storm-topology-as-nimbus storm-id blob-store)
@@ -897,7 +901,7 @@
storm-cluster-state (:storm-cluster-state nimbus)
^INimbus inimbus (:inimbus nimbus)
;; read all the topologies
- topology-ids (.active-storms storm-cluster-state)
+ topology-ids (.activeStorms storm-cluster-state)
topologies (into {} (for [tid topology-ids]
{tid (read-topology-details nimbus tid)}))
topologies (Topologies. topologies)
@@ -908,7 +912,7 @@
;; we exclude its assignment, meaning that all the slots occupied by its assignment
;; will be treated as free slot in the scheduler code.
(when (or (nil? scratch-topology-id) (not= tid scratch-topology-id))
- {tid (.assignment-info storm-cluster-state tid nil)})))
+ {tid (clojurify-assignment (.assignmentInfo storm-cluster-state tid nil))})))
;; make the new assignments for topologies
new-scheduler-assignments (compute-new-scheduler-assignments
nimbus
@@ -957,7 +961,7 @@
(log-debug "Assignment for " topology-id " hasn't changed")
(do
(log-message "Setting new assignment for topology id " topology-id ": " (pr-str assignment))
- (.set-assignment! storm-cluster-state topology-id assignment)
+ (.setAssignment storm-cluster-state topology-id (thriftify-assignment assignment))
)))
(->> new-assignments
(map (fn [[topology-id assignment]]
@@ -984,9 +988,9 @@
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
+ (.activateStorm storm-cluster-state
storm-id
- (StormBase. storm-name
+ (thriftify-storm-base (StormBase. storm-name
(current-time-secs)
{:type topology-initial-status}
(storm-conf TOPOLOGY-WORKERS)
@@ -994,7 +998,7 @@
(storm-conf TOPOLOGY-SUBMITTER-USER)
nil
nil
- {}))
+ {})))
(notify-topology-action-listener nimbus storm-name "activate")))
;; Master:
@@ -1046,10 +1050,10 @@
(set (.filterAndListKeys blob-store to-id))))
(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))
+ (let [heartbeat-ids (set (.heartbeatStorms storm-cluster-state))
+ error-ids (set (.errorTopologies storm-cluster-state))
code-ids (code-ids blob-store)
- assigned-ids (set (.active-storms storm-cluster-state))]
+ assigned-ids (set (.activeStorms storm-cluster-state))]
(set/difference (set/union heartbeat-ids error-ids code-ids) assigned-ids)
))
@@ -1113,7 +1117,7 @@
(try
(.deleteBlob blob-store key nimbus-subject)
(if (instance? LocalFsBlobStore blob-store)
- (.remove-blobstore-key! storm-cluster-state key))
+ (.removeBlobstoreKey storm-cluster-state key))
(catch Exception e
(log-message "Exception" e))))
@@ -1133,8 +1137,8 @@
(when-not (empty? to-cleanup-ids)
(doseq [id to-cleanup-ids]
(log-message "Cleaning up " id)
- (.teardown-heartbeats! storm-cluster-state id)
- (.teardown-topology-errors! storm-cluster-state id)
+ (.teardownHeartbeats storm-cluster-state id)
+ (.teardownTopologyErrors storm-cluster-state id)
(rmr (ConfigUtils/masterStormDistRoot conf id))
(blob-rm-topology-keys id blob-store storm-cluster-state)
(swap! (:heartbeats-cache nimbus) dissoc id)))))
@@ -1169,21 +1173,21 @@
(let [storm-cluster-state (:storm-cluster-state nimbus)
blob-store (:blob-store nimbus)
code-ids (set (code-ids blob-store))
- active-topologies (set (.active-storms storm-cluster-state))
+ active-topologies (set (.activeStorms 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)
+ (.removeStorm 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))))))
+ (.removeBlobstoreKey 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))
+ all-keys (set (.activeKeys 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)
@@ -1193,10 +1197,10 @@
(.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)))))
+ (.setupBlobstore 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)
+ (->> (apply clojurify-error (.errors storm-cluster-state storm-id component-id))
(map #(doto (ErrorInfo. (:error %) (:time-secs %))
(.set_host (:host %))
(.set_port (:port %))))))
@@ -1293,11 +1297,11 @@
blob-store (:blob-store nimbus)
renewers (:cred-renewers nimbus)
update-lock (:cred-update-lock nimbus)
- assigned-ids (set (.active-storms storm-cluster-state))]
+ assigned-ids (set (.activeStorms storm-cluster-state))]
(when-not (empty? assigned-ids)
(doseq [id assigned-ids]
(locking update-lock
- (let [orig-creds (.credentials storm-cluster-state id nil)
+ (let [orig-creds (clojurify-crdentials (.credentials storm-cluster-state id nil))
topology-conf (try-read-storm-conf (:conf nimbus) id blob-store)]
(if orig-creds
(let [new-creds (HashMap. orig-creds)]
@@ -1305,7 +1309,7 @@
(log-message "Renewing Creds For " id " with " renewer)
(.renew renewer new-creds (Collections/unmodifiableMap topology-conf)))
(when-not (= orig-creds new-creds)
- (.set-credentials! storm-cluster-state id new-creds topology-conf)
+ (.setCredentials storm-cluster-state id (thriftify-credentials new-creds) topology-conf)
))))))))
(log-message "not a leader skipping , credential renweal.")))
@@ -1370,11 +1374,11 @@
operation)
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)
+ base (clojurify-storm-base (.stormBase storm-cluster-state storm-id nil))
launch-time-secs (if base (:launch-time-secs base)
(throw
(NotAliveException. (str storm-id))))
- assignment (.assignment-info storm-cluster-state storm-id nil)
+ assignment (clojurify-assignment (.assignmentInfo storm-cluster-state storm-id nil))
beats (map-val :heartbeat (get @(:heartbeats-cache nimbus)
storm-id))
all-components (set (vals task->component))]
@@ -1388,16 +1392,16 @@
:task->component task->component
:base base}))
get-last-error (fn [storm-cluster-state storm-id component-id]
- (if-let [e (.last-error storm-cluster-state
+ (if-let [e (clojurify-error (.lastError storm-cluster-state
storm-id
- component-id)]
+ component-id))]
(doto (ErrorInfo. (:error e) (:time-secs e))
(.set_host (:host e))
(.set_port (:port e)))))]
(.prepare ^org.apache.storm.nimbus.ITopologyValidator (:validator nimbus) conf)
;add to nimbuses
- (.add-nimbus-host! (:storm-cluster-state nimbus) (.toHostPortString (:nimbus-host-port-info nimbus))
+ (.addNimbusHost (:storm-cluster-state nimbus) (.toHostPortString (:nimbus-host-port-info nimbus))
(NimbusSummary.
(.getHost (:nimbus-host-port-info nimbus))
(.getPort (:nimbus-host-port-info nimbus))
@@ -1413,7 +1417,7 @@
(setup-blobstore nimbus))
(when (is-leader nimbus :throw-exception false)
- (doseq [storm-id (.active-storms (:storm-cluster-state nimbus))]
+ (doseq [storm-id (.activeStorms (:storm-cluster-state nimbus))]
(transition! nimbus storm-id :startup)))
(schedule-recurring (:timer nimbus)
0
@@ -1520,12 +1524,12 @@
(locking (:submit-lock nimbus)
(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)
+ (.setCredentials storm-cluster-state storm-id (thriftify-credentials credentials) storm-conf)
(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)
+ (.setupHeatbeats storm-cluster-state storm-id)
+ (.setupBackpressure storm-cluster-state storm-id)
(notify-topology-action-listener nimbus storm-name "submitTopology")
(let [thrift-status->kw-status {TopologyInitialStatus/INACTIVE :inactive
TopologyInitialStatus/ACTIVE :active}]
@@ -1613,7 +1617,7 @@
(log-message "Nimbus setting debug to " enable? " for storm-name '" storm-name "' storm-id '" storm-id "' sampling pct '" spct "'"
(if (not (clojure.string/blank? component-id)) (str " component-id '" component-id "'")))
(locking (:submit-lock nimbus)
- (.update-storm! storm-cluster-state storm-id storm-base-updates))))
+ (.updateStorm storm-cluster-state (thriftify-storm-base storm-id storm-base-updates)))))
(^void setWorkerProfiler
[this ^String id ^ProfileRequest profileRequest]
@@ -1622,7 +1626,7 @@
storm-name (topology-conf TOPOLOGY-NAME)
_ (check-authorization! nimbus storm-name topology-conf "setWorkerProfiler")
storm-cluster-state (:storm-cluster-state nimbus)]
- (.set-worker-profile-request storm-cluster-state id profileRequest)))
+ (.setWorkerProfileRequest storm-cluster-state id profileRequest)))
(^List getComponentPendingProfileActions
[this ^String id ^String component_id ^ProfileAction action]
@@ -1635,7 +1639,7 @@
[(node->host node) port])
executor->node+port)
nodeinfos (stats/extract-nodeinfos-from-hb-for-comp executor->host+port task->component false component_id)
- all-pending-actions-for-topology (.get-topology-profile-requests storm-cluster-state id true)
+ all-pending-actions-for-topology (clojurify-profile-request (.getTopologyProfileRequests storm-cluster-state id true))
latest-profile-actions (remove nil? (map (fn [nodeInfo]
(->> all-pending-actions-for-topology
(filter #(and (= (:host nodeInfo) (.get_node (.get_nodeInfo %)))
@@ -1653,7 +1657,7 @@
storm-name (topology-conf TOPOLOGY-NAME)
_ (check-authorization! nimbus storm-name topology-conf "setLogConfig")
storm-cluster-state (:storm-cluster-state nimbus)
- merged-log-config (or (.topology-log-config storm-cluster-state id nil) (LogConfig.))
+ merged-log-config (or (.topologyLogConfig storm-cluster-state id nil) (LogConfig.))
named-loggers (.get_named_logger_level merged-log-config)]
(doseq [[_ level] named-loggers]
(.set_action level LogLevelAction/UNCHANGED))
@@ -1671,7 +1675,7 @@
(.containsKey named-loggers logger-name))
(.remove named-loggers logger-name))))))
(log-message "Setting log config for " storm-name ":" merged-log-config)
- (.set-topology-log-config! storm-cluster-state id merged-log-config)))
+ (.setTopologyLogConfig storm-cluster-state id merged-log-config)))
(uploadNewCredentials [this storm-name credentials]
(mark! nimbus:num-uploadNewCredentials-calls)
@@ -1680,7 +1684,7 @@
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))))
+ (locking (:cred-update-lock nimbus) (.setCredentials storm-cluster-state storm-id (thriftify-credentials creds) topology-conf))))
(beginFileUpload [this]
(mark! nimbus:num-beginFileUpload-calls)
@@ -1755,7 +1759,7 @@
storm-name (topology-conf TOPOLOGY-NAME)
_ (check-authorization! nimbus storm-name topology-conf "getLogConfig")
storm-cluster-state (:storm-cluster-state nimbus)
- log-config (.topology-log-config storm-cluster-state id nil)]
+ log-config (.topologyLogConfig storm-cluster-state id nil)]
(if log-config log-config (LogConfig.))))
(^String getTopologyConf [this ^String id]
@@ -1800,7 +1804,8 @@
(when-let [version (:version info)] (.set_version sup-sum version))
sup-sum))
nimbus-uptime ((:uptime nimbus))
- bases (topology-bases storm-cluster-state)
+ javabases (topology-bases storm-cluster-state)
+ bases (into {} (dofor [[id base] javabases][id (clojurify-storm-base base)]))
nimbuses (.nimbuses storm-cluster-state)
;;update the isLeader field for each nimbus summary
@@ -1812,7 +1817,7 @@
(.set_isLeader nimbus-summary (and (= leader-host (.get_host nimbus-summary)) (= leader-port (.get_port nimbus-summary))))))
topology-summaries (dofor [[id base] bases :when base]
- (let [assignment (.assignment-info storm-cluster-state id nil)
+ (let [assignment (clojurify-assignment (.assignmentInfo storm-cluster-state id nil))
topo-summ (TopologySummary. id
(:storm-name base)
(->> (:executor->node+port assignment)
@@ -1939,7 +1944,7 @@
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)))
+ (.setupBlobstore 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]
@@ -2019,8 +2024,8 @@
(.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))
+ (.removeBlobstoreKey (:storm-cluster-state nimbus) blob-key)
+ (.removeKeyVersion (:storm-cluster-state nimbus) blob-key))
(log-message "Deleted blob for key " blob-key)))
(^ListBlobsResult listBlobs [this ^String session]
@@ -2157,7 +2162,8 @@
(^TopologyHistoryInfo getTopologyHistory [this ^String user]
(let [storm-cluster-state (:storm-cluster-state nimbus)
- bases (topology-bases storm-cluster-state)
+ javabases (topology-bases storm-cluster-state)
+ bases (into {} (dofor [[id base] javabases][id (clojurify-storm-base base)]))
assigned-topology-ids (.assignments storm-cluster-state nil)
user-group-match-fn (fn [topo-id user conf]
(let [topology-conf (try-read-storm-conf conf topo-id (:blob-store nimbus))
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
index 337a1b4..079b221 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
@@ -19,11 +19,11 @@
[org.apache.storm.utils LocalState Time Utils ConfigUtils]
[org.apache.storm.daemon Shutdownable]
[org.apache.storm Constants]
- [org.apache.storm.cluster ClusterStateContext DaemonType]
+ [org.apache.storm.cluster ClusterStateContext DaemonType StormZkClusterState Cluster]
[java.net JarURLConnection]
[java.net URI]
[org.apache.commons.io FileUtils])
- (:use [org.apache.storm config util log timer local-state])
+ (:use [org.apache.storm config util log timer local-state converter])
(:import [org.apache.storm.generated AuthorizationException KeyNotFoundException WorkerResources])
(:import [org.apache.storm.utils NimbusLeaderNotFoundException VersionInfo])
(:import [java.nio.file Files StandardCopyOption])
@@ -33,7 +33,7 @@
(:use [org.apache.storm.daemon common])
(:require [org.apache.storm.command [healthcheck :as healthcheck]])
(:require [org.apache.storm.daemon [worker :as worker]]
- [org.apache.storm [process-simulator :as psim] [cluster :as cluster] [event :as event]]
+ [org.apache.storm [process-simulator :as psim] [event :as event]]
[clojure.set :as set])
(:import [org.apache.thrift.transport TTransportException])
(:import [org.apache.zookeeper data.ACL ZooDefs$Ids ZooDefs$Perms])
@@ -63,21 +63,22 @@
(->>
(dofor [sid storm-ids]
(let [recorded-version (:version (get assignment-versions sid))]
- (if-let [assignment-version (.assignment-version storm-cluster-state sid callback)]
+ (if-let [assignment-version (.assignmentVersion storm-cluster-state sid callback)]
(if (= assignment-version recorded-version)
{sid (get assignment-versions sid)}
- {sid (.assignment-info-with-version storm-cluster-state sid callback)})
+ {sid (.assignmentInfoWithVersion storm-cluster-state sid callback)})
{sid nil})))
(apply merge)
(filter-val not-nil?))
new-profiler-actions
(->>
(dofor [sid (distinct storm-ids)]
- (if-let [topo-profile-actions (.get-topology-profile-requests storm-cluster-state sid false)]
+
+ (if-let [topo-profile-actions (into [] (for [request (.getTopologyProfileRequests storm-cluster-state sid false)] (clojurify-profile-request request)))]
{sid topo-profile-actions}))
(apply merge))]
-
- {:assignments (into {} (for [[k v] new-assignments] [k (:data v)]))
+
+ {:assignments (into {} (for [[k v] new-assignments] [k (clojurify-assignment (:data v))]))
:profiler-actions new-profiler-actions
:versions new-assignments})))
@@ -316,11 +317,9 @@
:uptime (uptime-computer)
:version STORM-VERSION
:worker-thread-pids-atom (atom {})
- :storm-cluster-state (cluster/mk-storm-cluster-state conf :acls (when
- (Utils/isZkAuthenticationConfiguredStormServer
- conf)
- SUPERVISOR-ZK-ACLS)
- :context (ClusterStateContext. DaemonType/SUPERVISOR))
+ :storm-cluster-state (Cluster/mkStormClusterState conf (when (Utils/isZkAuthenticationConfiguredStormServer conf)
+ SUPERVISOR-ZK-ACLS)
+ (ClusterStateContext. DaemonType/SUPERVISOR))
:local-state (ConfigUtils/supervisorState conf)
:supervisor-id (.getSupervisorId isupervisor)
:assignment-id (.getAssignmentId isupervisor)
@@ -675,7 +674,7 @@
(defn- delete-topology-profiler-action [storm-cluster-state storm-id profile-action]
(log-message "Deleting profiler action.." profile-action)
- (.delete-topology-profile-requests storm-cluster-state storm-id profile-action))
+ (.deleteTopologyProfileRequests storm-cluster-state storm-id (thriftify-profile-request profile-action)))
(defnk launch-profiler-action-for-worker
"Launch profiler action for a worker"
@@ -743,7 +742,7 @@
action-on-exit (fn [exit-code]
(log-message log-prefix " profile-action exited for code: " exit-code)
(if (and (= exit-code 0) stop?)
- (delete-topology-profiler-action storm-cluster-state storm-id pro-action)))
+ (delete-topology-profiler-action storm-cluster-state storm-id (thriftify-profile-request pro-action))))
command (->> command (map str) (filter (complement empty?)))]
(try
@@ -776,10 +775,10 @@
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!
+ heartbeat-fn (fn [] (.supervisorHeartbeat
(:storm-cluster-state supervisor)
(:supervisor-id supervisor)
- (->SupervisorInfo (current-time-secs)
+ (thriftify-supervisor-info (->SupervisorInfo (current-time-secs)
(:my-hostname supervisor)
(:assignment-id supervisor)
(keys @(:curr-assignment supervisor))
@@ -788,7 +787,7 @@
(conf SUPERVISOR-SCHEDULER-META)
((:uptime supervisor))
(:version supervisor)
- (mk-supervisor-capacities conf))))]
+ (mk-supervisor-capacities conf)))))]
(heartbeat-fn)
;; should synchronize supervisor so it doesn't launch anything after being down (optimization)
[13/27] storm git commit: Merge branch 'master' into ClusterUtils
Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
index 8df5885,0000000..17c8641
mode 100644,000000..100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
@@@ -1,664 -1,0 +1,687 @@@
+/**
+ * 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.*;
+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.ZKStateChangedCallback;
+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.Watcher;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
++import java.io.PrintWriter;
++import java.io.StringWriter;
+import java.security.NoSuchAlgorithmException;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+
+public class StormClusterStateImpl implements IStormClusterState {
+
+ private static Logger LOG = LoggerFactory.getLogger(StormClusterStateImpl.class);
+
+ private IStateStorage stateStorage;
+
+ 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 StormClusterStateImpl(IStateStorage StateStorage, List<ACL> acls, ClusterStateContext context, boolean solo) throws Exception {
+
+ this.stateStorage = StateStorage;
+ this.solo = solo;
++ this.acls = acls;
+
+ 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.stateStorage.register(new ZKStateChangedCallback() {
+
+ public void changed(Watcher.Event.EventType type, String path) {
+ 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(ClusterUtils.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(ClusterUtils.SUPERVISORS_ROOT)) {
+ issueCallback(supervisorsCallback);
+ } else if (root.equals(ClusterUtils.BLOBSTORE_ROOT)) {
+ issueCallback(blobstoreCallback);
+ } else if (root.equals(ClusterUtils.STORMS_ROOT) && size > 1) {
+ issueMapCallback(stormBaseCallback, toks.get(1));
+ } else if (root.equals(ClusterUtils.CREDENTIALS_ROOT) && size > 1) {
+ issueMapCallback(credentialsCallback, toks.get(1));
+ } else if (root.equals(ClusterUtils.LOGCONFIG_ROOT) && size > 1) {
+ issueMapCallback(logConfigCallback, toks.get(1));
+ } else if (root.equals(ClusterUtils.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;
+ }
+
+ });
+
+ String[] pathlist = { ClusterUtils.ASSIGNMENTS_SUBTREE, ClusterUtils.STORMS_SUBTREE, ClusterUtils.SUPERVISORS_SUBTREE, ClusterUtils.WORKERBEATS_SUBTREE,
+ ClusterUtils.ERRORS_SUBTREE, ClusterUtils.BLOBSTORE_SUBTREE, ClusterUtils.NIMBUSES_SUBTREE, ClusterUtils.LOGCONFIG_SUBTREE };
+ for (String path : pathlist) {
+ this.stateStorage.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 stateStorage.get_children(ClusterUtils.ASSIGNMENTS_SUBTREE, callback != null);
+ }
+
+ @Override
+ public Assignment assignmentInfo(String stormId, IFn callback) {
+ if (callback != null) {
+ assignmentInfoCallback.put(stormId, callback);
+ }
+ byte[] serialized = stateStorage.get_data(ClusterUtils.assignmentPath(stormId), callback != null);
+ return ClusterUtils.maybeDeserialize(serialized, Assignment.class);
+ }
+
+ @Override
+ public APersistentMap assignmentInfoWithVersion(String stormId, IFn callback) {
+ if (callback != null) {
+ assignmentInfoWithVersionCallback.put(stormId, callback);
+ }
+ Assignment assignment = null;
+ Integer version = 0;
+ APersistentMap aPersistentMap = stateStorage.get_data_with_version(ClusterUtils.assignmentPath(stormId), callback != null);
+ if (aPersistentMap != null) {
+ assignment = ClusterUtils.maybeDeserialize((byte[]) aPersistentMap.get(RT.keyword(null, "data")), Assignment.class);
+ version = (Integer) aPersistentMap.get(RT.keyword(null, "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 stateStorage.get_version(ClusterUtils.assignmentPath(stormId), callback != null);
+ }
+
+ // blobstore state
+ @Override
+ public List<String> blobstoreInfo(String blobKey) {
+ String path = ClusterUtils.blobstorePath(blobKey);
+ stateStorage.sync_path(path);
+ return stateStorage.get_children(path, false);
+ }
+
+ @Override
+ public List nimbuses() {
+ List<NimbusSummary> nimbusSummaries = new ArrayList<>();
+ List<String> nimbusIds = stateStorage.get_children(ClusterUtils.NIMBUSES_SUBTREE, false);
+ for (String nimbusId : nimbusIds) {
+ byte[] serialized = stateStorage.get_data(ClusterUtils.nimbusPath(nimbusId), false);
+ NimbusSummary nimbusSummary = ClusterUtils.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.
+ stateStorage.delete_node(ClusterUtils.nimbusPath(nimbusId));
+ stateStorage.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");
+ stateStorage.set_ephemeral_node(ClusterUtils.nimbusPath(nimbusId), Utils.serialize(nimbusSummary), acls);
+ }
+
+ }
+ });
+
+ stateStorage.set_ephemeral_node(ClusterUtils.nimbusPath(nimbusId), Utils.serialize(nimbusSummary), acls);
+ }
+
+ @Override
+ public List<String> activeStorms() {
+ return stateStorage.get_children(ClusterUtils.STORMS_SUBTREE, false);
+ }
+
+ @Override
+ public StormBase stormBase(String stormId, IFn callback) {
+ if (callback != null) {
+ stormBaseCallback.put(stormId, callback);
+ }
+ return ClusterUtils.maybeDeserialize(stateStorage.get_data(ClusterUtils.stormPath(stormId), callback != null), StormBase.class);
+ }
+
+ @Override
+ public ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, String node, Long port) {
+ byte[] bytes = stateStorage.get_worker_hb(ClusterUtils.workerbeatPath(stormId, node, port), false);
+ return ClusterUtils.maybeDeserialize(bytes, ClusterWorkerHeartbeat.class);
+
+ }
+
+ @Override
+ public List<ProfileRequest> getWorkerProfileRequests(String stormId, NodeInfo nodeInfo, boolean isThrift) {
+ List<ProfileRequest> requests = new ArrayList<>();
+ List<ProfileRequest> profileRequests = getTopologyProfileRequests(stormId, isThrift);
+ for (ProfileRequest profileRequest : profileRequests) {
+ NodeInfo nodeInfo1 = profileRequest.get_nodeInfo();
+ if (nodeInfo1.equals(nodeInfo))
+ requests.add(profileRequest);
+ }
+ return requests;
+ }
+
+ @Override
+ public List<ProfileRequest> getTopologyProfileRequests(String stormId, boolean isThrift) {
+ List<ProfileRequest> profileRequests = new ArrayList<>();
+ String path = ClusterUtils.profilerConfigPath(stormId);
+ if (stateStorage.node_exists(path, false)) {
+ List<String> strs = stateStorage.get_children(path, false);
+ for (String str : strs) {
+ String childPath = path + ClusterUtils.ZK_SEPERATOR + str;
+ byte[] raw = stateStorage.get_data(childPath, false);
+ ProfileRequest request = ClusterUtils.maybeDeserialize(raw, ProfileRequest.class);
+ if (request != null)
+ profileRequests.add(request);
+ }
+ }
+ return profileRequests;
+ }
+
+ @Override
+ public void setWorkerProfileRequest(String stormId, ProfileRequest profileRequest) {
+ ProfileAction profileAction = profileRequest.get_action();
+ String host = profileRequest.get_nodeInfo().get_node();
+ Long port = profileRequest.get_nodeInfo().get_port_iterator().next();
+ String path = ClusterUtils.profilerConfigPath(stormId, host, port, profileAction);
+ stateStorage.set_data(path, Utils.serialize(profileRequest), acls);
+ }
+
+ @Override
+ public void deleteTopologyProfileRequests(String stormId, ProfileRequest profileRequest) {
+ ProfileAction profileAction = profileRequest.get_action();
+ String host = profileRequest.get_nodeInfo().get_node();
+ Long port = profileRequest.get_nodeInfo().get_port_iterator().next();
+ String path = ClusterUtils.profilerConfigPath(stormId, host, port, profileAction);
+ stateStorage.delete_node(path);
+ }
+
- // need to take executor->node+port in explicitly so that we don't run into a situation where a
- // long dead worker with a skewed clock overrides all the timestamps. By only checking heartbeats
- // with an assigned node+port, and only reading executors from that heartbeat that are actually assigned,
- // we avoid situations like that
++ /**
++ * need to take executor->node+port in explicitly so that we don't run into a situation where a long dead worker with a skewed clock overrides all the
++ * timestamps. By only checking heartbeats with an assigned node+port, and only reading executors from that heartbeat that are actually assigned, we avoid
++ * situations like that
++ *
++ * @param stormId
++ * @param executorNodePort
++ * @return
++ */
+ @Override
- public Map<ExecutorInfo, ClusterWorkerHeartbeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort) {
- Map<ExecutorInfo, ClusterWorkerHeartbeat> executorWhbs = new HashMap<>();
++ public Map<ExecutorInfo, APersistentMap> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort) {
++ Map<ExecutorInfo, APersistentMap> executorWhbs = new HashMap<>();
+
+ Map<NodeInfo, List<List<Long>>> nodePortExecutors = ClusterUtils.reverseMap(executorNodePort);
+
+ for (Map.Entry<NodeInfo, List<List<Long>>> entry : nodePortExecutors.entrySet()) {
+
+ String node = entry.getKey().get_node();
+ Long port = entry.getKey().get_port_iterator().next();
+ ClusterWorkerHeartbeat whb = getWorkerHeartbeat(stormId, node, port);
+ List<ExecutorInfo> executorInfoList = new ArrayList<>();
+ for (List<Long> list : entry.getValue()) {
+ executorInfoList.add(new ExecutorInfo(list.get(0).intValue(), list.get(list.size() - 1).intValue()));
+ }
+ if (whb != null)
+ executorWhbs.putAll(ClusterUtils.convertExecutorBeats(executorInfoList, whb));
+ }
+ return executorWhbs;
+ }
+
+ @Override
+ public List<String> supervisors(IFn callback) {
+ if (callback != null) {
+ supervisorsCallback.set(callback);
+ }
+ return stateStorage.get_children(ClusterUtils.SUPERVISORS_SUBTREE, callback != null);
+ }
+
+ @Override
+ public SupervisorInfo supervisorInfo(String supervisorId) {
+ String path = ClusterUtils.supervisorPath(supervisorId);
+ return ClusterUtils.maybeDeserialize(stateStorage.get_data(path, false), SupervisorInfo.class);
+ }
+
+ @Override
+ public void setupHeatbeats(String stormId) {
+ stateStorage.mkdirs(ClusterUtils.workerbeatStormRoot(stormId), acls);
+ }
+
+ @Override
+ public void teardownHeartbeats(String stormId) {
+ try {
+ stateStorage.delete_worker_hb(ClusterUtils.workerbeatStormRoot(stormId));
+ } catch (Exception e) {
+ if (Zookeeper.exceptionCause(KeeperException.class, e)) {
+ // do nothing
+ LOG.warn("Could not teardown heartbeats for {}.", stormId);
+ } else {
+ throw e;
+ }
+ }
+ }
+
+ @Override
+ public void teardownTopologyErrors(String stormId) {
+ try {
+ stateStorage.delete_node(ClusterUtils.errorStormRoot(stormId));
+ } catch (Exception e) {
+ if (Zookeeper.exceptionCause(KeeperException.class, e)) {
+ // do nothing
+ LOG.warn("Could not teardown errors for {}.", stormId);
+ } else {
+ throw e;
+ }
+ }
+ }
+
+ @Override
+ public List<String> heartbeatStorms() {
+ return stateStorage.get_worker_hb_children(ClusterUtils.WORKERBEATS_SUBTREE, false);
+ }
+
+ @Override
+ public List<String> errorTopologies() {
+ return stateStorage.get_children(ClusterUtils.ERRORS_SUBTREE, false);
+ }
+
+ @Override
+ public void setTopologyLogConfig(String stormId, LogConfig logConfig) {
+ stateStorage.set_data(ClusterUtils.logConfigPath(stormId), Utils.serialize(logConfig), acls);
+ }
+
+ @Override
+ public LogConfig topologyLogConfig(String stormId, IFn cb) {
+ String path = ClusterUtils.logConfigPath(stormId);
+ return ClusterUtils.maybeDeserialize(stateStorage.get_data(path, cb != null), LogConfig.class);
+ }
+
+ @Override
+ public void workerHeartbeat(String stormId, String node, Long port, ClusterWorkerHeartbeat info) {
+ if (info != null) {
+ String path = ClusterUtils.workerbeatPath(stormId, node, port);
+ stateStorage.set_worker_hb(path, Utils.serialize(info), acls);
+ }
+ }
+
+ @Override
+ public void removeWorkerHeartbeat(String stormId, String node, Long port) {
+ String path = ClusterUtils.workerbeatPath(stormId, node, port);
+ stateStorage.delete_worker_hb(path);
+ }
+
+ @Override
+ public void supervisorHeartbeat(String supervisorId, SupervisorInfo info) {
+ String path = ClusterUtils.supervisorPath(supervisorId);
+ stateStorage.set_ephemeral_node(path, Utils.serialize(info), acls);
+ }
+
- // if znode exists and to be not on?, delete; if exists and on?, do nothing;
- // if not exists and to be on?, create; if not exists and not on?, do nothing;
++ /**
++ * if znode exists and to be not on?, delete; if exists and on?, do nothing; if not exists and to be on?, create; if not exists and not on?, do nothing;
++ *
++ * @param stormId
++ * @param node
++ * @param port
++ * @param on
++ */
+ @Override
+ public void workerBackpressure(String stormId, String node, Long port, boolean on) {
+ String path = ClusterUtils.backpressurePath(stormId, node, port);
+ boolean existed = stateStorage.node_exists(path, false);
+ if (existed) {
+ if (on == false)
+ stateStorage.delete_node(path);
+
+ } else {
+ if (on == true) {
+ stateStorage.set_ephemeral_node(path, null, acls);
+ }
+ }
+ }
+
- // if the backpresure/storm-id dir is empty, this topology has throttle-on, otherwise not.
++ /**
++ * if the backpresure/storm-id dir is empty, this topology has throttle-on, otherwise not.
++ *
++ * @param stormId
++ * @param callback
++ * @return
++ */
+ @Override
+ public boolean topologyBackpressure(String stormId, IFn callback) {
+ if (callback != null) {
+ backPressureCallback.put(stormId, callback);
+ }
+ String path = ClusterUtils.backpressureStormRoot(stormId);
+ List<String> childrens = stateStorage.get_children(path, callback != null);
+ return childrens.size() > 0;
+
+ }
+
+ @Override
+ public void setupBackpressure(String stormId) {
+ stateStorage.mkdirs(ClusterUtils.backpressureStormRoot(stormId), acls);
+ }
+
+ @Override
+ public void removeWorkerBackpressure(String stormId, String node, Long port) {
+ stateStorage.delete_node(ClusterUtils.backpressurePath(stormId, node, port));
+ }
+
+ @Override
+ public void activateStorm(String stormId, StormBase stormBase) {
+ String path = ClusterUtils.stormPath(stormId);
+ stateStorage.set_data(path, Utils.serialize(stormBase), acls);
+ }
+
- // To update this function due to APersistentMap/APersistentSet is clojure's structure
++ /**
++ * To update this function due to APersistentMap/APersistentSet is clojure's structure
++ *
++ * @param stormId
++ * @param newElems
++ */
+ @Override
+ public void updateStorm(String stormId, StormBase newElems) {
+
+ StormBase stormBase = stormBase(stormId, null);
+ if (stormBase.get_component_executors() != null) {
+
+ Map<String, Integer> newComponentExecutors = new HashMap<>();
+ Map<String, Integer> componentExecutors = newElems.get_component_executors();
+ // componentExecutors maybe be APersistentMap, which don't support "put"
+ for (Map.Entry<String, Integer> entry : componentExecutors.entrySet()) {
+ newComponentExecutors.put(entry.getKey(), entry.getValue());
+ }
+ for (Map.Entry<String, Integer> entry : stormBase.get_component_executors().entrySet()) {
+ if (!componentExecutors.containsKey(entry.getKey())) {
+ newComponentExecutors.put(entry.getKey(), entry.getValue());
+ }
+ }
+ if (newComponentExecutors.size() > 0)
+ newElems.set_component_executors(newComponentExecutors);
+ }
+
+ Map<String, DebugOptions> ComponentDebug = new HashMap<>();
+ Map<String, DebugOptions> oldComponentDebug = stormBase.get_component_debug();
+
+ Map<String, DebugOptions> newComponentDebug = newElems.get_component_debug();
+ /// oldComponentDebug.keySet()/ newComponentDebug.keySet() maybe be APersistentSet, which don't support addAll
+ Set<String> debugOptionsKeys = new HashSet<>();
+ debugOptionsKeys.addAll(oldComponentDebug.keySet());
+ debugOptionsKeys.addAll(newComponentDebug.keySet());
+ for (String key : debugOptionsKeys) {
+ boolean enable = false;
+ double samplingpct = 0;
+ if (oldComponentDebug.containsKey(key)) {
+ enable = oldComponentDebug.get(key).is_enable();
+ samplingpct = oldComponentDebug.get(key).get_samplingpct();
+ }
+ if (newComponentDebug.containsKey(key)) {
+ enable = newComponentDebug.get(key).is_enable();
+ samplingpct += newComponentDebug.get(key).get_samplingpct();
+ }
+ DebugOptions debugOptions = new DebugOptions();
+ debugOptions.set_enable(enable);
+ debugOptions.set_samplingpct(samplingpct);
+ ComponentDebug.put(key, debugOptions);
+ }
+ if (ComponentDebug.size() > 0) {
+ newElems.set_component_debug(ComponentDebug);
+ }
+
+ if (StringUtils.isBlank(newElems.get_name())) {
+ newElems.set_name(stormBase.get_name());
+ }
+ if (newElems.get_status() == null) {
+ newElems.set_status(stormBase.get_status());
+ }
+ if (newElems.get_num_workers() == 0) {
+ newElems.set_num_workers(stormBase.get_num_workers());
+ }
+ if (newElems.get_launch_time_secs() == 0) {
+ newElems.set_launch_time_secs(stormBase.get_launch_time_secs());
+ }
+ if (StringUtils.isBlank(newElems.get_owner())) {
+ newElems.set_owner(stormBase.get_owner());
+ }
+ if (newElems.get_topology_action_options() == null) {
+ newElems.set_topology_action_options(stormBase.get_topology_action_options());
+ }
+ if (newElems.get_status() == null) {
+ newElems.set_status(stormBase.get_status());
+ }
+ stateStorage.set_data(ClusterUtils.stormPath(stormId), Utils.serialize(newElems), acls);
+ }
+
+ @Override
+ public void removeStormBase(String stormId) {
+ stateStorage.delete_node(ClusterUtils.stormPath(stormId));
+ }
+
+ @Override
+ public void setAssignment(String stormId, Assignment info) {
+ stateStorage.set_data(ClusterUtils.assignmentPath(stormId), Utils.serialize(info), acls);
+ }
+
+ @Override
+ public void setupBlobstore(String key, NimbusInfo nimbusInfo, Integer versionInfo) {
+ String path = ClusterUtils.blobstorePath(key) + ClusterUtils.ZK_SEPERATOR + nimbusInfo.toHostPortString() + "-" + versionInfo;
+ LOG.info("set-path: {}", path);
+ stateStorage.mkdirs(ClusterUtils.blobstorePath(key), acls);
+ stateStorage.delete_node_blobstore(ClusterUtils.blobstorePath(key), nimbusInfo.toHostPortString());
+ stateStorage.set_ephemeral_node(path, null, acls);
+ }
+
+ @Override
+ public List<String> activeKeys() {
+ return stateStorage.get_children(ClusterUtils.BLOBSTORE_SUBTREE, false);
+ }
+
+ // blobstore state
+ @Override
+ public List<String> blobstore(IFn callback) {
+ if (callback != null) {
+ blobstoreCallback.set(callback);
+ }
+ stateStorage.sync_path(ClusterUtils.BLOBSTORE_SUBTREE);
+ return stateStorage.get_children(ClusterUtils.BLOBSTORE_SUBTREE, callback != null);
+
+ }
+
+ @Override
+ public void removeStorm(String stormId) {
+ stateStorage.delete_node(ClusterUtils.assignmentPath(stormId));
+ stateStorage.delete_node(ClusterUtils.credentialsPath(stormId));
+ stateStorage.delete_node(ClusterUtils.logConfigPath(stormId));
+ stateStorage.delete_node(ClusterUtils.profilerConfigPath(stormId));
+ removeStormBase(stormId);
+ }
+
+ @Override
+ public void removeBlobstoreKey(String blobKey) {
+ LOG.debug("remove key {}", blobKey);
+ stateStorage.delete_node(ClusterUtils.blobstorePath(blobKey));
+ }
+
+ @Override
+ public void removeKeyVersion(String blobKey) {
+ stateStorage.delete_node(ClusterUtils.blobstoreMaxKeySequenceNumberPath(blobKey));
+ }
+
+ @Override
- public void reportError(String stormId, String componentId, String node, Long port, String error) {
++ public void reportError(String stormId, String componentId, String node, Long port, Throwable error) {
+
+ String path = ClusterUtils.errorPath(stormId, componentId);
+ String lastErrorPath = ClusterUtils.lastErrorPath(stormId, componentId);
- ErrorInfo errorInfo = new ErrorInfo(error, Time.currentTimeSecs());
++ ErrorInfo errorInfo = new ErrorInfo(ClusterUtils.StringifyError(error), Time.currentTimeSecs());
+ errorInfo.set_host(node);
+ errorInfo.set_port(port.intValue());
+ byte[] serData = Utils.serialize(errorInfo);
+ stateStorage.mkdirs(path, acls);
+ stateStorage.create_sequential(path + ClusterUtils.ZK_SEPERATOR + "e", serData, acls);
+ stateStorage.set_data(lastErrorPath, serData, acls);
+ List<String> childrens = stateStorage.get_children(path, false);
+
+ Collections.sort(childrens, new Comparator<String>() {
+ public int compare(String arg0, String arg1) {
+ return Long.compare(Long.parseLong(arg0.substring(1)), Long.parseLong(arg1.substring(1)));
+ }
+ });
+
+ while (childrens.size() > 10) {
+ stateStorage.delete_node(path + ClusterUtils.ZK_SEPERATOR + childrens.remove(0));
+ }
+ }
+
+ @Override
+ public List<ErrorInfo> errors(String stormId, String componentId) {
+ List<ErrorInfo> errorInfos = new ArrayList<>();
+ try {
+ String path = ClusterUtils.errorPath(stormId, componentId);
+ if (stateStorage.node_exists(path, false)) {
+ List<String> childrens = stateStorage.get_children(path, false);
+ for (String child : childrens) {
+ String childPath = path + ClusterUtils.ZK_SEPERATOR + child;
+ ErrorInfo errorInfo = ClusterUtils.maybeDeserialize(stateStorage.get_data(childPath, false), ErrorInfo.class);
+ if (errorInfo != null)
+ errorInfos.add(errorInfo);
+ }
+ }
+ Collections.sort(errorInfos, new Comparator<ErrorInfo>() {
+ public int compare(ErrorInfo arg0, ErrorInfo arg1) {
+ return Integer.compare(arg1.get_error_time_secs(), arg0.get_error_time_secs());
+ }
+ });
+ } catch (Exception e) {
+ throw Utils.wrapInRuntime(e);
+ }
+
+ return errorInfos;
+ }
+
+ @Override
+ public ErrorInfo lastError(String stormId, String componentId) {
+
+ String path = ClusterUtils.lastErrorPath(stormId, componentId);
+ if (stateStorage.node_exists(path, false)) {
+ ErrorInfo errorInfo = ClusterUtils.maybeDeserialize(stateStorage.get_data(path, false), ErrorInfo.class);
+ return errorInfo;
+ }
+
+ return null;
+ }
+
+ @Override
+ public void setCredentials(String stormId, Credentials creds, Map topoConf) throws NoSuchAlgorithmException {
+ List<ACL> aclList = ClusterUtils.mkTopoOnlyAcls(topoConf);
+ String path = ClusterUtils.credentialsPath(stormId);
+ stateStorage.set_data(path, Utils.serialize(creds), aclList);
+
+ }
+
+ @Override
+ public Credentials credentials(String stormId, IFn callback) {
+ if (callback != null) {
+ credentialsCallback.put(stormId, callback);
+ }
+ String path = ClusterUtils.credentialsPath(stormId);
+ return ClusterUtils.maybeDeserialize(stateStorage.get_data(path, callback != null), Credentials.class);
+
+ }
+
+ @Override
+ public void disconnect() {
+ stateStorage.unregister(stateId);
+ if (solo)
+ stateStorage.close();
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
index f3b9253,0000000..956c20e
mode 100644,000000..100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
@@@ -1,36 -1,0 +1,36 @@@
+/**
+ * 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 org.apache.storm.utils.Utils;
+import org.apache.zookeeper.data.ACL;
+
+import java.util.List;
+
- public class ZKStateStorageFactory implements StateStorageFactory{
++public class ZKStateStorageFactory implements StateStorageFactory {
+
+ @Override
+ public IStateStorage mkStore(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) {
+ try {
+ return new ZKStateStorage(config, auth_conf, acls, context);
- }catch (Exception e){
++ } catch (Exception e) {
+ throw Utils.wrapInRuntime(e);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
index af0e8f3,34f3665..20d6deb
--- a/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
+++ b/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
@@@ -60,6 -60,6 +60,10 @@@ public class PacemakerClient implement
private StormBoundedExponentialBackoffRetry backoff = new StormBoundedExponentialBackoffRetry(100, 5000, 20);
private int retryTimes = 0;
++ //the constructor is invoked by pacemaker-state-factory-test
++ public PacemakerClient() {
++ bootstrap = new ClientBootstrap();
++ }
public PacemakerClient(Map config) {
String host = (String)config.get(Config.PACEMAKER_HOST);
@@@ -157,6 -157,7 +161,7 @@@
public String secretKey() {
return secret;
}
-
++ public HBMessage checkCaptured() {return null;}
public HBMessage send(HBMessage m) {
waitUntilReady();
LOG.debug("Sending message: {}", m.toString());
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/test/clj/org/apache/storm/cluster_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/cluster_test.clj
index 39adb9e,b146cb0..22c1f80
--- a/storm-core/test/clj/org/apache/storm/cluster_test.clj
+++ b/storm-core/test/clj/org/apache/storm/cluster_test.clj
@@@ -22,11 -22,11 +22,11 @@@
(:import [org.mockito Mockito])
(:import [org.mockito.exceptions.base MockitoAssertionError])
(:import [org.apache.curator.framework CuratorFramework CuratorFrameworkFactory CuratorFrameworkFactory$Builder])
- (:import [org.apache.storm.utils Utils TestUtils ZookeeperAuthInfo ConfigUtils])
+ (:import [org.apache.storm.utils Time Utils ZookeeperAuthInfo ConfigUtils])
- (:import [org.apache.storm.cluster ClusterState])
+ (:import [org.apache.storm.cluster IStateStorage ZKStateStorage ClusterStateContext StormClusterStateImpl ClusterUtils])
(:import [org.apache.storm.zookeeper Zookeeper])
- (:import [org.apache.storm.testing.staticmocking MockedZookeeper])
- (:require [org.apache.storm [zookeeper :as zk]])
+ (:import [org.apache.storm.callback ZKStateChangedCallback])
+ (:import [org.apache.storm.testing.staticmocking MockedZookeeper MockedCluster])
(:require [conjure.core])
(:use [conjure core])
(:use [clojure test])
@@@ -39,14 -39,18 +39,18 @@@
(defn mk-state
([zk-port] (let [conf (mk-config zk-port)]
- (mk-distributed-cluster-state conf :auth-conf conf)))
+ (ClusterUtils/mkStateStorage conf conf nil (ClusterStateContext.))))
([zk-port cb]
- (let [ret (mk-state zk-port)]
- (.register ret cb)
- ret )))
+ (let [ret (mk-state zk-port)]
+ (.register ret cb)
+ ret)))
-(defn mk-storm-state [zk-port] (mk-storm-cluster-state (mk-config zk-port)))
+(defn mk-storm-state [zk-port] (ClusterUtils/mkStormClusterState (mk-config zk-port) nil (ClusterStateContext.)))
+ (defn barr
+ [& vals]
+ (byte-array (map byte vals)))
+
(deftest test-basics
(with-inprocess-zookeeper zk-port
(let [state (mk-state zk-port)]
@@@ -242,27 -244,26 +246,32 @@@
(is (.contains (:error error) target))
)))
++(defn- stringify-error [error]
++ (let [result (java.io.StringWriter.)
++ printer (java.io.PrintWriter. result)]
++ (.printStackTrace error printer)
++ (.toString result)))
(deftest test-storm-cluster-state-errors
(with-inprocess-zookeeper zk-port
(with-simulated-time
(let [state (mk-storm-state zk-port)]
- (.reportError state "a" "1" (local-hostname) 6700 (stringify-error (RuntimeException.)))
- (.report-error state "a" "1" (Utils/localHostname) 6700 (RuntimeException.))
++ (.reportError state "a" "1" (Utils/localHostname) 6700 (RuntimeException.))
(validate-errors! state "a" "1" ["RuntimeException"])
(advance-time-secs! 1)
- (.reportError state "a" "1" (local-hostname) 6700 (stringify-error (IllegalArgumentException.)))
- (.report-error state "a" "1" (Utils/localHostname) 6700 (IllegalArgumentException.))
++ (.reportError state "a" "1" (Utils/localHostname) 6700 (IllegalArgumentException.))
(validate-errors! state "a" "1" ["IllegalArgumentException" "RuntimeException"])
(doseq [i (range 10)]
- (.reportError state "a" "2" (local-hostname) 6700 (stringify-error (RuntimeException.)))
- (.report-error state "a" "2" (Utils/localHostname) 6700 (RuntimeException.))
++ (.reportError state "a" "2" (Utils/localHostname) 6700 (RuntimeException.))
(advance-time-secs! 2))
(validate-errors! state "a" "2" (repeat 10 "RuntimeException"))
(doseq [i (range 5)]
- (.reportError state "a" "2" (local-hostname) 6700 (stringify-error (IllegalArgumentException.)))
- (.report-error state "a" "2" (Utils/localHostname) 6700 (IllegalArgumentException.))
++ (.reportError state "a" "2" (Utils/localHostname) 6700 (IllegalArgumentException.))
(advance-time-secs! 2))
(validate-errors! state "a" "2" (concat (repeat 5 "IllegalArgumentException")
- (repeat 5 "RuntimeException")
- ))
+ (repeat 5 "RuntimeException")
+ ))
+
(.disconnect state)
))))
@@@ -300,12 -301,12 +309,12 @@@
(. (Mockito/when (.connectString builder (Mockito/anyString))) (thenReturn builder))
(. (Mockito/when (.connectionTimeoutMs builder (Mockito/anyInt))) (thenReturn builder))
(. (Mockito/when (.sessionTimeoutMs builder (Mockito/anyInt))) (thenReturn builder))
- (TestUtils/testSetupBuilder builder (str zk-port "/") conf (ZookeeperAuthInfo. conf))
+ (Utils/testSetupBuilder builder (str zk-port "/") conf (ZookeeperAuthInfo. conf))
(is (nil?
- (try
- (. (Mockito/verify builder) (authorization "digest" (.getBytes (conf STORM-ZOOKEEPER-AUTH-PAYLOAD))))
- (catch MockitoAssertionError e
- e)))))))
+ (try
+ (. (Mockito/verify builder) (authorization "digest" (.getBytes (conf STORM-ZOOKEEPER-AUTH-PAYLOAD))))
+ (catch MockitoAssertionError e
+ e)))))))
(deftest test-storm-state-callbacks
;; TODO finish
@@@ -313,17 -314,15 +322,17 @@@
(deftest test-cluster-state-default-acls
(testing "The default ACLs are empty."
- (let [zk-mock (Mockito/mock Zookeeper)]
+ (let [zk-mock (Mockito/mock Zookeeper)
+ curator-frameworke (reify CuratorFramework (^void close [this] nil))]
;; No need for when clauses because we just want to return nil
(with-open [_ (MockedZookeeper. zk-mock)]
- (stubbing [zk/mk-client (reify CuratorFramework (^void close [this] nil))]
- (mk-distributed-cluster-state {})
- (.mkdirsImpl (Mockito/verify zk-mock (Mockito/times 1)) (Mockito/any) (Mockito/anyString) (Mockito/eq nil)))))
- (stubbing [mk-distributed-cluster-state (reify ClusterState
- (register [this callback] nil)
- (mkdirs [this path acls] nil))]
- (mk-storm-cluster-state {})
- (verify-call-times-for mk-distributed-cluster-state 1)
- (verify-first-call-args-for-indices mk-distributed-cluster-state [4] nil))))
+ (. (Mockito/when (.mkClientImpl zk-mock (Mockito/anyMap) (Mockito/anyList) (Mockito/any) (Mockito/anyString) (Mockito/any) (Mockito/anyMap))) (thenReturn curator-frameworke))
+ (ClusterUtils/mkStateStorage {} nil nil (ClusterStateContext.))
+ (.mkdirsImpl (Mockito/verify zk-mock (Mockito/times 1)) (Mockito/any) (Mockito/anyString) (Mockito/eq nil))))
+ (let [distributed-state-storage (reify IStateStorage
+ (register [this callback] nil)
+ (mkdirs [this path acls] nil))
+ cluster-utils (Mockito/mock ClusterUtils)]
+ (with-open [mocked-cluster (MockedCluster. cluster-utils)]
- (. (Mockito/when (mkStateStorageImpl cluster-utils (Mockito/any) (Mockito/any) (Mockito/eq nil) (Mockito/any))) (thenReturn distributed-state-storage))
++ (. (Mockito/when (.mkStateStorageImpl cluster-utils (Mockito/any) (Mockito/any) (Mockito/eq nil) (Mockito/any))) (thenReturn distributed-state-storage))
+ (ClusterUtils/mkStormClusterState {} nil (ClusterStateContext.))))))
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/test/clj/org/apache/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/nimbus_test.clj
index 772a232,70cb885..09c4371
--- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj
+++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
@@@ -23,27 -23,33 +23,36 @@@
[org.apache.storm.nimbus InMemoryTopologyActionNotifier])
(:import [org.apache.storm.testing.staticmocking MockedZookeeper])
(:import [org.apache.storm.scheduler INimbus])
+ (:import [org.mockito Mockito])
+ (:import [org.mockito.exceptions.base MockitoAssertionError])
(:import [org.apache.storm.nimbus ILeaderElector NimbusInfo])
- (:import [org.apache.storm.testing.staticmocking MockedConfigUtils MockedCluster])
++ (:import [org.apache.storm.testing.staticmocking MockedCluster])
(:import [org.apache.storm.generated Credentials NotAliveException SubmitOptions
TopologyInitialStatus TopologyStatus AlreadyAliveException KillOptions RebalanceOptions
InvalidTopologyException AuthorizationException
LogConfig LogLevel LogLevelAction])
(:import [java.util HashMap])
(:import [java.io File])
- (:import [org.apache.storm.utils Time Utils ConfigUtils])
+ (:import [org.apache.storm.utils Time Utils Utils$UptimeComputer ConfigUtils IPredicate]
+ [org.apache.storm.utils.staticmocking ConfigUtilsInstaller UtilsInstaller])
(:import [org.apache.storm.zookeeper Zookeeper])
- (:import [org.apache.commons.io FileUtils])
+ (:import [org.apache.commons.io FileUtils]
+ [org.json.simple JSONValue])
- (:use [org.apache.storm testing MockAutoCred util config log timer zookeeper])
+ (:import [org.apache.storm.cluster StormClusterStateImpl ClusterStateContext ClusterUtils])
+ (:use [org.apache.storm testing MockAutoCred util config log timer converter])
(:use [org.apache.storm.daemon common])
(:require [conjure.core])
(:require [org.apache.storm
- [thrift :as thrift]
- [cluster :as cluster]])
+ [thrift :as thrift]])
(:use [conjure core]))
+ (defn- from-json
+ [^String str]
+ (if str
+ (clojurify-structure
+ (JSONValue/parse str))
+ nil))
+
(defn storm-component->task-info [cluster storm-name]
(let [storm-id (get-storm-id (:storm-cluster-state cluster) storm-name)
nimbus (:nimbus cluster)]
@@@ -72,8 -80,8 +83,8 @@@
(defn storm-num-workers [state storm-name]
(let [storm-id (get-storm-id state storm-name)
- assignment (.assignment-info state storm-id nil)]
+ assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
- (count (reverse-map (:executor->node+port assignment)))
+ (count (clojurify-structure (Utils/reverseMap (:executor->node+port assignment))))
))
(defn topology-nodes [state storm-name]
@@@ -95,9 -103,11 +106,11 @@@
set
)))
+ ;TODO: when translating this function, don't call map-val, but instead use an inline for loop.
+ ; map-val is a temporary kluge for clojure.
(defn topology-node-distribution [state storm-name]
(let [storm-id (get-storm-id state storm-name)
- assignment (.assignment-info state storm-id nil)]
+ assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
(->> assignment
:executor->node+port
vals
@@@ -124,19 -134,18 +137,18 @@@
(defn do-executor-heartbeat [cluster storm-id executor]
(let [state (:storm-cluster-state cluster)
- executor->node+port (:executor->node+port (.assignment-info state storm-id nil))
+ executor->node+port (:executor->node+port (clojurify-assignment (.assignmentInfo state storm-id nil)))
[node port] (get executor->node+port executor)
- curr-beat (.get-worker-heartbeat state storm-id node port)
+ curr-beat (clojurify-zk-worker-hb (.getWorkerHeartbeat state storm-id node port))
stats (:executor-stats curr-beat)]
- (.worker-heartbeat! state storm-id node port
- {:storm-id storm-id :time-secs (Time/currentTimeSecs) :uptime 10 :executor-stats (merge stats {executor (stats/render-stats! (stats/mk-bolt-stats 20))})}
+ (.workerHeartbeat state storm-id node port
- (thriftify-zk-worker-hb {:storm-id storm-id :time-secs (current-time-secs) :uptime 10 :executor-stats (merge stats {executor (stats/render-stats! (stats/mk-bolt-stats 20))})})
++ (thriftify-zk-worker-hb {:storm-id storm-id :time-secs (Time/currentTimeSecs) :uptime 10 :executor-stats (merge stats {executor (stats/render-stats! (stats/mk-bolt-stats 20))})})
)))
(defn slot-assignments [cluster storm-id]
(let [state (:storm-cluster-state cluster)
- assignment (.assignment-info state storm-id nil)]
+ assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
- (reverse-map (:executor->node+port assignment))
- ))
+ (clojurify-structure (Utils/reverseMap (:executor->node+port assignment)))))
(defn task-ids [cluster storm-id]
(let [nimbus (:nimbus cluster)]
@@@ -146,8 -155,10 +158,10 @@@
(defn topology-executors [cluster storm-id]
(let [state (:storm-cluster-state cluster)
- assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
- (keys (:executor->node+port assignment))
- assignment (.assignment-info state storm-id nil)
- ret-keys (keys (:executor->node+port assignment))
++ assignment (clojurify-assignment (.assignmentInfo state storm-id nil))
++ ret-keys (keys (:executor->node+port assignment))
+ _ (log-message "ret-keys: " (pr-str ret-keys)) ]
+ ret-keys
))
(defn check-distribution [items distribution]
@@@ -1350,23 -1399,27 +1402,27 @@@
NIMBUS-THRIFT-PORT 6666})
expected-acls nimbus/NIMBUS-ZK-ACLS
fake-inimbus (reify INimbus (getForcedScheduler [this] nil))
+ fake-cu (proxy [ConfigUtils] []
- (nimbusTopoHistoryStateImpl [conf] nil))
++ (nimbusTopoHistoryStateImpl [conf] nil))
+ fake-utils (proxy [Utils] []
+ (newInstanceImpl [_])
+ (makeUptimeComputer [] (proxy [Utils$UptimeComputer] []
- (upTime [] 0))))]
++ (upTime [] 0))))
+ cluster-utils (Mockito/mock ClusterUtils)]
- (with-open [_ (proxy [MockedConfigUtils] []
- (nimbusTopoHistoryStateImpl [conf] nil))
+ (with-open [_ (ConfigUtilsInstaller. fake-cu)
+ _ (UtilsInstaller. fake-utils)
zk-le (MockedZookeeper. (proxy [Zookeeper] []
- (zkLeaderElectorImpl [conf] nil)))]
+ (zkLeaderElectorImpl [conf] nil)))
+ mocked-cluster (MockedCluster. cluster-utils)]
(stubbing [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
- mk-timer nil
- nimbus/mk-scheduler nil]
- (nimbus/nimbus-data auth-conf fake-inimbus)
- (verify-call-times-for cluster/mk-storm-cluster-state 1)
- (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
- expected-acls))))))
+ 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-scheduler nil]
+ (nimbus/nimbus-data auth-conf fake-inimbus)
+ (.mkStormClusterStateImpl (Mockito/verify cluster-utils (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
+ )))))
(deftest test-file-bogus-download
(with-local-cluster [cluster :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0 TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
@@@ -1397,9 -1450,9 +1453,9 @@@
STORM-CLUSTER-MODE "local"
STORM-ZOOKEEPER-PORT zk-port
STORM-LOCAL-DIR nimbus-dir}))
- (bind cluster-state (cluster/mk-storm-cluster-state conf))
+ (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
(bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
- (sleep-secs 1)
+ (Time/sleepSecs 1)
(bind topology (thrift/mk-topology
{"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
{}))
@@@ -1429,10 -1482,10 +1485,10 @@@
STORM-ZOOKEEPER-PORT zk-port
STORM-LOCAL-DIR nimbus-dir
NIMBUS-TOPOLOGY-ACTION-NOTIFIER-PLUGIN (.getName InMemoryTopologyActionNotifier)}))
- (bind cluster-state (cluster/mk-storm-cluster-state conf))
+ (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
(bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
(bind notifier (InMemoryTopologyActionNotifier.))
- (sleep-secs 1)
+ (Time/sleepSecs 1)
(bind topology (thrift/mk-topology
{"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
{}))
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
index 1a7bd2c,0925237..1c45266
--- a/storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
+++ b/storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
@@@ -19,10 -20,8 +19,11 @@@
(:import [org.apache.storm.generated
HBExecutionException HBNodes HBRecords
HBServerMessageType HBMessage HBMessageData HBPulse]
- [org.apache.storm.cluster ClusterStateContext PaceMakerStateStorageFactory]
- [org.apache.storm.cluster ClusterStateContext]
- [org.mockito Mockito Matchers]))
++ [org.apache.storm.cluster ClusterStateContext PaceMakerStateStorageFactory PaceMakerStateStorage]
+ [org.mockito Mockito Matchers])
+(:import [org.mockito.exceptions.base MockitoAssertionError])
++(:import [org.apache.storm.pacemaker PacemakerClient])
+(:import [org.apache.storm.testing.staticmocking MockedPaceMakerStateStorageFactory]))
(defn- string-to-bytes [string]
(byte-array (map int string)))
@@@ -30,26 -29,24 +31,23 @@@
(defn- bytes-to-string [bytez]
(apply str (map char bytez)))
--(defprotocol send-capture
-- (send [this something])
-- (check-captured [this]))
--
(defn- make-send-capture [response]
(let [captured (atom nil)]
-- (reify send-capture
-- (send [this something] (reset! captured something) response)
-- (check-captured [this] @captured))))
-
-(defmacro with-mock-pacemaker-client-and-state [client state response & body]
- `(let [~client (make-send-capture ~response)]
- (stubbing [psf/makeZKState nil
- psf/makeClient ~client]
- (let [~state (psf/-mkState nil nil nil nil (ClusterStateContext.))]
++ (proxy [PacemakerClient] []
++ (send [m] (reset! captured m) response)
++ (checkCaptured [] @captured))))
+
+(defmacro with-mock-pacemaker-client-and-state [client state pacefactory mock response & body]
+ `(let [~client (make-send-capture ~response)
+ ~pacefactory (Mockito/mock PaceMakerStateStorageFactory)]
+
+ (with-open [~mock (MockedPaceMakerStateStorageFactory. ~pacefactory)]
+ (. (Mockito/when (.initZKstateImpl ~pacefactory (Mockito/any) (Mockito/any) (Mockito/anyList) (Mockito/any))) (thenReturn nil))
+ (. (Mockito/when (.initMakeClientImpl ~pacefactory (Mockito/any))) (thenReturn ~client))
- (let [~state (.mkStore ~pacefactory nil nil nil (ClusterStateContext.))]
++ (let [~state (PaceMakerStateStorage. (PaceMakerStateStorageFactory/initMakeClient nil)
++ (PaceMakerStateStorageFactory/initZKstate nil nil nil nil))]
~@body))))
-
(deftest pacemaker_state_set_worker_hb
(testing "set_worker_hb"
(with-mock-pacemaker-client-and-state
@@@ -57,7 -54,7 +55,7 @@@
(HBMessage. HBServerMessageType/SEND_PULSE_RESPONSE nil)
(.set_worker_hb state "/foo" (string-to-bytes "data") nil)
-- (let [sent (.check-captured client)
++ (let [sent (.checkCaptured client)
pulse (.get_pulse (.get_data sent))]
(is (= (.get_type sent) HBServerMessageType/SEND_PULSE))
(is (= (.get_id pulse) "/foo"))
@@@ -65,13 -62,13 +63,12 @@@
(testing "set_worker_hb"
(with-mock-pacemaker-client-and-state
- client state
+ client state pacefactory mock
(HBMessage. HBServerMessageType/SEND_PULSE nil)
- (is (thrown? RuntimeException
- (is (thrown? HBExecutionException
-- (.set_worker_hb state "/foo" (string-to-bytes "data") nil))))))
++ (is (thrown? RuntimeException
++ (.set_worker_hb state "/foo" (string-to-bytes "data") nil))))))
--
(deftest pacemaker_state_delete_worker_hb
(testing "delete_worker_hb"
@@@ -80,74 -77,74 +77,75 @@@
(HBMessage. HBServerMessageType/DELETE_PATH_RESPONSE nil)
(.delete_worker_hb state "/foo/bar")
-- (let [sent (.check-captured client)]
++ (let [sent (.checkCaptured client)]
(is (= (.get_type sent) HBServerMessageType/DELETE_PATH))
(is (= (.get_path (.get_data sent)) "/foo/bar")))))
-- (testing "delete_worker_hb"
-- (with-mock-pacemaker-client-and-state
- client state pacefactory mock
- client state
-- (HBMessage. HBServerMessageType/DELETE_PATH nil)
--
- (is (thrown? RuntimeException
- (is (thrown? HBExecutionException
-- (.delete_worker_hb state "/foo/bar"))))))
++ (testing "delete_worker_hb"
++ (with-mock-pacemaker-client-and-state
++ client state pacefactory mock
++ (HBMessage. HBServerMessageType/DELETE_PATH nil)
++
++ (is (thrown? RuntimeException
++ (.delete_worker_hb state "/foo/bar"))))))
(deftest pacemaker_state_get_worker_hb
(testing "get_worker_hb"
(with-mock-pacemaker-client-and-state
- client state
+ client state pacefactory mock
(HBMessage. HBServerMessageType/GET_PULSE_RESPONSE
-- (HBMessageData/pulse
-- (doto (HBPulse.)
-- (.set_id "/foo")
-- (.set_details (string-to-bytes "some data")))))
++ (HBMessageData/pulse
++ (doto (HBPulse.)
++ (.set_id "/foo")
++ (.set_details (string-to-bytes "some data")))))
(.get_worker_hb state "/foo" false)
-- (let [sent (.check-captured client)]
++ (let [sent (.checkCaptured client)]
(is (= (.get_type sent) HBServerMessageType/GET_PULSE))
(is (= (.get_path (.get_data sent)) "/foo")))))
(testing "get_worker_hb - fail (bad response)"
(with-mock-pacemaker-client-and-state
- client state
+ client state pacefactory mock
(HBMessage. HBServerMessageType/GET_PULSE nil)
--
- (is (thrown? HBExecutionException
- (.get_worker_hb state "/foo" false)))))
-
++
+ (is (thrown? RuntimeException
- (.get_worker_hb state "/foo" false)))))
-
++ (.get_worker_hb state "/foo" false)))))
++
(testing "get_worker_hb - fail (bad data)"
(with-mock-pacemaker-client-and-state
- client state
+ client state pacefactory mock
(HBMessage. HBServerMessageType/GET_PULSE_RESPONSE nil)
--
- (is (thrown? HBExecutionException
- (.get_worker_hb state "/foo" false))))))
++
+ (is (thrown? RuntimeException
- (.get_worker_hb state "/foo" false))))))
++ (.get_worker_hb state "/foo" false))))))
(deftest pacemaker_state_get_worker_hb_children
(testing "get_worker_hb_children"
(with-mock-pacemaker-client-and-state
- client state
+ client state pacefactory mock
(HBMessage. HBServerMessageType/GET_ALL_NODES_FOR_PATH_RESPONSE
-- (HBMessageData/nodes
-- (HBNodes. [])))
++ (HBMessageData/nodes
++ (HBNodes. [])))
(.get_worker_hb_children state "/foo" false)
-- (let [sent (.check-captured client)]
++ (let [sent (.checkCaptured client)]
(is (= (.get_type sent) HBServerMessageType/GET_ALL_NODES_FOR_PATH))
(is (= (.get_path (.get_data sent)) "/foo")))))
(testing "get_worker_hb_children - fail (bad response)"
(with-mock-pacemaker-client-and-state
- client state
+ client state pacefactory mock
(HBMessage. HBServerMessageType/DELETE_PATH nil)
- (is (thrown? HBExecutionException
- (.get_worker_hb_children state "/foo" false)))))
+ (is (thrown? RuntimeException
- (.get_worker_hb_children state "/foo" false)))))
++ (.get_worker_hb_children state "/foo" false)))))
-- (testing "get_worker_hb_children - fail (bad data)"
++ (testing "get_worker_hb_children - fail (bad data)"
(with-mock-pacemaker-client-and-state
- client state
+ client state pacefactory mock
(HBMessage. HBServerMessageType/GET_ALL_NODES_FOR_PATH_RESPONSE nil)
- ;need been update due to HBExecutionException
-
- (is (thrown? HBExecutionException
- (.get_worker_hb_children state "/foo" false))))))
++
+ (is (thrown? RuntimeException
- (.get_worker_hb_children state "/foo" false))))))
++ (.get_worker_hb_children state "/foo" false))))))
++
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/test/clj/org/apache/storm/supervisor_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/supervisor_test.clj
index b89b7bb,9c31ddf..3ebdbcd
--- a/storm-core/test/clj/org/apache/storm/supervisor_test.clj
+++ b/storm-core/test/clj/org/apache/storm/supervisor_test.clj
@@@ -21,20 -21,20 +21,23 @@@
(:require [clojure [string :as string] [set :as set]])
(:import [org.apache.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter TestPlannerSpout])
(:import [org.apache.storm.scheduler ISupervisor])
- (:import [org.apache.storm.utils ConfigUtils])
+ (:import [org.apache.storm.utils Time Utils$UptimeComputer ConfigUtils])
(:import [org.apache.storm.generated RebalanceOptions])
- (:import [org.apache.storm.testing.staticmocking MockedConfigUtils MockedCluster])
- (:import [org.mockito Matchers Mockito])
++ (:import [org.apache.storm.testing.staticmocking MockedCluster])
(:import [java.util UUID])
- (:import [org.mockito Mockito])
++ (:import [org.mockito Mockito Matchers])
+ (:import [org.mockito.exceptions.base MockitoAssertionError])
(:import [java.io File])
(:import [java.nio.file Files])
- (:import [org.apache.storm.cluster StormClusterStateImpl ClusterStateContext ClusterUtils])
- (:import [org.apache.storm.utils Utils IPredicate]
++ (:import [org.apache.storm.utils Utils IPredicate])
++ (:import [org.apache.storm.cluster StormClusterStateImpl ClusterStateContext ClusterUtils]
+ [org.apache.storm.utils.staticmocking ConfigUtilsInstaller
+ UtilsInstaller])
(:import [java.nio.file.attribute FileAttribute])
- (:use [org.apache.storm config testing util timer log])
+ (:use [org.apache.storm config testing util timer log converter])
(:use [org.apache.storm.daemon common])
(:require [org.apache.storm.daemon [worker :as worker] [supervisor :as supervisor]]
- [org.apache.storm [thrift :as thrift] [cluster :as cluster]])
+ [org.apache.storm [thrift :as thrift]])
(:use [conjure core])
(:require [clojure.java.io :as io]))
@@@ -43,9 -43,10 +46,10 @@@
[cluster supervisor-id port]
(let [state (:storm-cluster-state cluster)
slot-assigns (for [storm-id (.assignments state nil)]
- (let [executors (-> (.assignment-info state storm-id nil)
+ (let [executors (-> (clojurify-assignment (.assignmentInfo state storm-id nil))
:executor->node+port
- reverse-map
+ (Utils/reverseMap)
+ clojurify-structure
(get [supervisor-id port] ))]
(when executors [storm-id executors])
))
@@@ -565,198 -632,203 +635,201 @@@
fake-isupervisor (reify ISupervisor
(getSupervisorId [this] nil)
(getAssignmentId [this] nil))
+ fake-cu (proxy [ConfigUtils] []
+ (supervisorStateImpl [conf] nil)
+ (supervisorLocalDirImpl [conf] nil))
+ fake-utils (proxy [Utils] []
+ (localHostnameImpl [] nil)
+ (makeUptimeComputer [] (proxy [Utils$UptimeComputer] []
- (upTime [] 0))))]
++ (upTime [] 0))))
+ cluster-utils (Mockito/mock ClusterUtils)]
- (with-open [_ (proxy [MockedConfigUtils] []
- (supervisorStateImpl [conf] nil)
- (supervisorLocalDirImpl [conf] nil))
+ (with-open [_ (ConfigUtilsInstaller. fake-cu)
- _ (UtilsInstaller. fake-utils)]
- (stubbing [cluster/mk-storm-cluster-state nil
- mk-timer nil]
++ _ (UtilsInstaller. fake-utils)
+ mocked-cluster (MockedCluster. cluster-utils)]
- (stubbing [uptime-computer nil
- ; cluster/mk-storm-cluster-state nil
- local-hostname nil
- mk-timer nil]
(supervisor/supervisor-data auth-conf nil fake-isupervisor)
- (.mkStormClusterStateImpl (Mockito/verify cluster-utils (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
- ; (verify-call-times-for cluster/mk-storm-cluster-state 1)
- ; (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
- ; expected-acls)
- )))))
- (verify-call-times-for cluster/mk-storm-cluster-state 1)
- (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
- expected-acls)))))
++ (.mkStormClusterStateImpl (Mockito/verify cluster-utils (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))))))
- (deftest test-write-log-metadata
- (testing "supervisor writes correct data to logs metadata file"
- (let [exp-owner "alice"
- exp-worker-id "42"
- exp-storm-id "0123456789"
- exp-port 4242
- exp-logs-users ["bob" "charlie" "daryl"]
- exp-logs-groups ["read-only-group" "special-group"]
- storm-conf {TOPOLOGY-SUBMITTER-USER "alice"
- TOPOLOGY-USERS ["charlie" "bob"]
- TOPOLOGY-GROUPS ["special-group"]
- LOGS-GROUPS ["read-only-group"]
- LOGS-USERS ["daryl"]}
- exp-data {TOPOLOGY-SUBMITTER-USER exp-owner
- "worker-id" exp-worker-id
- LOGS-USERS exp-logs-users
- LOGS-GROUPS exp-logs-groups}
- conf {}]
- (mocking [supervisor/write-log-metadata-to-yaml-file!]
- (supervisor/write-log-metadata! storm-conf exp-owner exp-worker-id
- exp-storm-id exp-port conf)
- (verify-called-once-with-args supervisor/write-log-metadata-to-yaml-file!
- exp-storm-id exp-port exp-data conf)))))
+ (deftest test-write-log-metadata
+ (testing "supervisor writes correct data to logs metadata file"
+ (let [exp-owner "alice"
+ exp-worker-id "42"
+ exp-storm-id "0123456789"
+ exp-port 4242
+ exp-logs-users ["bob" "charlie" "daryl"]
+ exp-logs-groups ["read-only-group" "special-group"]
+ storm-conf {TOPOLOGY-SUBMITTER-USER "alice"
+ TOPOLOGY-USERS ["charlie" "bob"]
+ TOPOLOGY-GROUPS ["special-group"]
+ LOGS-GROUPS ["read-only-group"]
+ LOGS-USERS ["daryl"]}
+ exp-data {TOPOLOGY-SUBMITTER-USER exp-owner
+ "worker-id" exp-worker-id
+ LOGS-USERS exp-logs-users
+ LOGS-GROUPS exp-logs-groups}
+ conf {}]
+ (mocking [supervisor/write-log-metadata-to-yaml-file!]
+ (supervisor/write-log-metadata! storm-conf exp-owner exp-worker-id
+ exp-storm-id exp-port conf)
+ (verify-called-once-with-args supervisor/write-log-metadata-to-yaml-file!
+ exp-storm-id exp-port exp-data conf)))))
- (deftest test-worker-launcher-requires-user
- (testing "worker-launcher throws on blank user"
- (mocking [launch-process]
- (is (thrown-cause-with-msg? java.lang.IllegalArgumentException
- #"(?i).*user cannot be blank.*"
- (supervisor/worker-launcher {} nil ""))))))
+ (deftest test-worker-launcher-requires-user
+ (testing "worker-launcher throws on blank user"
+ (let [utils-proxy (proxy [Utils] []
+ (launchProcessImpl [& _] nil))]
+ (with-open [_ (UtilsInstaller. utils-proxy)]
+ (is (try
+ (supervisor/worker-launcher {} nil "")
+ false
+ (catch Throwable t
+ (and (re-matches #"(?i).*user cannot be blank.*" (.getMessage t))
+ (Utils/exceptionCauseIsInstanceOf java.lang.IllegalArgumentException t)))))))))
- (defn found? [sub-str input-str]
- (if (string? input-str)
- (contrib-str/substring? sub-str (str input-str))
- (boolean (some #(contrib-str/substring? sub-str %) input-str))))
+ (defn found? [sub-str input-str]
+ (if (string? input-str)
+ (contrib-str/substring? sub-str (str input-str))
+ (boolean (some #(contrib-str/substring? sub-str %) input-str))))
- (defn not-found? [sub-str input-str]
+ (defn not-found? [sub-str input-str]
(complement (found? sub-str input-str)))
- (deftest test-substitute-childopts-happy-path-string
- (testing "worker-launcher replaces ids in childopts"
- (let [worker-id "w-01"
- topology-id "s-01"
- port 9999
- mem-onheap 512
- childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log -Xms256m -Xmx%HEAP-MEM%m"
- expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
- childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
- (is (= expected-childopts childopts-with-ids)))))
+ (deftest test-substitute-childopts-happy-path-string
+ (testing "worker-launcher replaces ids in childopts"
+ (let [worker-id "w-01"
+ topology-id "s-01"
+ port 9999
+ mem-onheap 512
+ childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log -Xms256m -Xmx%HEAP-MEM%m"
+ expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
+ childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+ (is (= expected-childopts childopts-with-ids)))))
- (deftest test-substitute-childopts-happy-path-list
- (testing "worker-launcher replaces ids in childopts"
- (let [worker-id "w-01"
- topology-id "s-01"
- port 9999
- mem-onheap 512
- childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m" "-Xmx%HEAP-MEM%m")
- expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
- childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
- (is (= expected-childopts childopts-with-ids)))))
+ (deftest test-substitute-childopts-happy-path-list
+ (testing "worker-launcher replaces ids in childopts"
+ (let [worker-id "w-01"
+ topology-id "s-01"
+ port 9999
+ mem-onheap 512
+ childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m" "-Xmx%HEAP-MEM%m")
+ expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
+ childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+ (is (= expected-childopts childopts-with-ids)))))
- (deftest test-substitute-childopts-happy-path-list-arraylist
- (testing "worker-launcher replaces ids in childopts"
- (let [worker-id "w-01"
- topology-id "s-01"
- port 9999
- mem-onheap 512
- childopts '["-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m" "-Xmx%HEAP-MEM%m"]
- expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
- childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
- (is (= expected-childopts childopts-with-ids)))))
+ (deftest test-substitute-childopts-happy-path-list-arraylist
+ (testing "worker-launcher replaces ids in childopts"
+ (let [worker-id "w-01"
+ topology-id "s-01"
+ port 9999
+ mem-onheap 512
+ childopts '["-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m" "-Xmx%HEAP-MEM%m"]
+ expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
+ childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+ (is (= expected-childopts childopts-with-ids)))))
- (deftest test-substitute-childopts-topology-id-alone
- (testing "worker-launcher replaces ids in childopts"
- (let [worker-id "w-01"
- topology-id "s-01"
- port 9999
- mem-onheap 512
- childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%TOPOLOGY-ID%.log"
- expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-s-01.log")
- childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
- (is (= expected-childopts childopts-with-ids)))))
+ (deftest test-substitute-childopts-topology-id-alone
+ (testing "worker-launcher replaces ids in childopts"
+ (let [worker-id "w-01"
+ topology-id "s-01"
+ port 9999
+ mem-onheap 512
+ childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%TOPOLOGY-ID%.log"
+ expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-s-01.log")
+ childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+ (is (= expected-childopts childopts-with-ids)))))
- (deftest test-substitute-childopts-no-keys
- (testing "worker-launcher has no ids to replace in childopts"
- (let [worker-id "w-01"
- topology-id "s-01"
- port 9999
- mem-onheap 512
- childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log"
- expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log")
- childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
- (is (= expected-childopts childopts-with-ids)))))
+ (deftest test-substitute-childopts-no-keys
+ (testing "worker-launcher has no ids to replace in childopts"
+ (let [worker-id "w-01"
+ topology-id "s-01"
+ port 9999
+ mem-onheap 512
+ childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log"
+ expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log")
+ childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+ (is (= expected-childopts childopts-with-ids)))))
- (deftest test-substitute-childopts-nil-childopts
- (testing "worker-launcher has nil childopts"
- (let [worker-id "w-01"
- topology-id "s-01"
- port 9999
- mem-onheap 512
- childopts nil
- expected-childopts nil
- childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
- (is (= expected-childopts childopts-with-ids)))))
+ (deftest test-substitute-childopts-nil-childopts
+ (testing "worker-launcher has nil childopts"
+ (let [worker-id "w-01"
+ topology-id "s-01"
+ port 9999
+ mem-onheap 512
+ childopts nil
+ expected-childopts nil
+ childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+ (is (= expected-childopts childopts-with-ids)))))
- (deftest test-substitute-childopts-nil-ids
- (testing "worker-launcher has nil ids"
- (let [worker-id nil
- topology-id "s-01"
- port 9999
- mem-onheap 512
- childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log"
- expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01--9999.log")
- childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
- (is (= expected-childopts childopts-with-ids)))))
+ (deftest test-substitute-childopts-nil-ids
+ (testing "worker-launcher has nil ids"
+ (let [worker-id nil
+ topology-id "s-01"
+ port 9999
+ mem-onheap 512
+ childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log"
+ expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01--9999.log")
+ childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+ (is (= expected-childopts childopts-with-ids)))))
- (deftest test-retry-read-assignments
- (with-simulated-time-local-cluster [cluster
- :supervisors 0
- :ports-per-supervisor 2
- :daemon-conf {ConfigUtils/NIMBUS_DO_NOT_REASSIGN true
- NIMBUS-MONITOR-FREQ-SECS 10
- TOPOLOGY-MESSAGE-TIMEOUT-SECS 30
- TOPOLOGY-ACKER-EXECUTORS 0}]
- (letlocals
- (bind sup1 (add-supervisor cluster :id "sup1" :ports [1 2 3 4]))
- (bind topology1 (thrift/mk-topology
- {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)}
- {}))
- (bind topology2 (thrift/mk-topology
- {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)}
- {}))
- (bind state (:storm-cluster-state cluster))
- (bind changed (capture-changed-workers
- (submit-mocked-assignment
- (:nimbus cluster)
- (:storm-cluster-state cluster)
- "topology1"
- {TOPOLOGY-WORKERS 2}
- topology1
- {1 "1"
- 2 "1"}
- {[1 1] ["sup1" 1]
- [2 2] ["sup1" 2]}
- {["sup1" 1] [0.0 0.0 0.0]
- ["sup1" 2] [0.0 0.0 0.0]
- })
- (submit-mocked-assignment
- (:nimbus cluster)
- (:storm-cluster-state cluster)
- "topology2"
- {TOPOLOGY-WORKERS 2}
- topology2
- {1 "1"
- 2 "1"}
- {[1 1] ["sup1" 1]
- [2 2] ["sup1" 2]}
- {["sup1" 1] [0.0 0.0 0.0]
- ["sup1" 2] [0.0 0.0 0.0]
- })
- ;; Instead of sleeping until topology is scheduled, rebalance topology so mk-assignments is called.
- (.rebalance (:nimbus cluster) "topology1" (doto (RebalanceOptions.) (.set_wait_secs 0)))
- ))
- (is (empty? (:launched changed)))
- (bind options (RebalanceOptions.))
- (.set_wait_secs options 0)
- (bind changed (capture-changed-workers
- (.rebalance (:nimbus cluster) "topology2" options)
- (advance-cluster-time cluster 10)
- (heartbeat-workers cluster "sup1" [1 2 3 4])
- (advance-cluster-time cluster 10)
- ))
- (validate-launched-once (:launched changed)
- {"sup1" [1 2]}
- (get-storm-id (:storm-cluster-state cluster) "topology1"))
- (validate-launched-once (:launched changed)
- {"sup1" [3 4]}
- (get-storm-id (:storm-cluster-state cluster) "topology2"))
- )))
+ (deftest test-retry-read-assignments
+ (with-simulated-time-local-cluster [cluster
+ :supervisors 0
+ :ports-per-supervisor 2
+ :daemon-conf {ConfigUtils/NIMBUS_DO_NOT_REASSIGN true
+ NIMBUS-MONITOR-FREQ-SECS 10
+ TOPOLOGY-MESSAGE-TIMEOUT-SECS 30
+ TOPOLOGY-ACKER-EXECUTORS 0}]
+ (letlocals
+ (bind sup1 (add-supervisor cluster :id "sup1" :ports [1 2 3 4]))
+ (bind topology1 (thrift/mk-topology
+ {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)}
+ {}))
+ (bind topology2 (thrift/mk-topology
+ {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)}
+ {}))
+ (bind state (:storm-cluster-state cluster))
+ (bind changed (capture-changed-workers
+ (submit-mocked-assignment
+ (:nimbus cluster)
+ (:storm-cluster-state cluster)
+ "topology1"
+ {TOPOLOGY-WORKERS 2}
+ topology1
+ {1 "1"
+ 2 "1"}
+ {[1 1] ["sup1" 1]
+ [2 2] ["sup1" 2]}
+ {["sup1" 1] [0.0 0.0 0.0]
+ ["sup1" 2] [0.0 0.0 0.0]
+ })
+ (submit-mocked-assignment
+ (:nimbus cluster)
+ (:storm-cluster-state cluster)
+ "topology2"
+ {TOPOLOGY-WORKERS 2}
+ topology2
+ {1 "1"
+ 2 "1"}
+ {[1 1] ["sup1" 1]
+ [2 2] ["sup1" 2]}
+ {["sup1" 1] [0.0 0.0 0.0]
+ ["sup1" 2] [0.0 0.0 0.0]
+ })
+ ;; Instead of sleeping until topology is scheduled, rebalance topology so mk-assignments is called.
+ (.rebalance (:nimbus cluster) "topology1" (doto (RebalanceOptions.) (.set_wait_secs 0)))
+ ))
+ (is (empty? (:launched changed)))
+ (bind options (RebalanceOptions.))
+ (.set_wait_secs options 0)
+ (bind changed (capture-changed-workers
+ (.rebalance (:nimbus cluster) "topology2" options)
+ (advance-cluster-time cluster 10)
+ (heartbeat-workers cluster "sup1" [1 2 3 4])
+ (advance-cluster-time cluster 10)
+ ))
+ (validate-launched-once (:launched changed)
+ {"sup1" [1 2]}
+ (get-storm-id (:storm-cluster-state cluster) "topology1"))
+ (validate-launched-once (:launched changed)
+ {"sup1" [3 4]}
+ (get-storm-id (:storm-cluster-state cluster) "topology2"))
- ))))
++ )))
[19/27] storm git commit: Merge branch 'master' into ClusterUtils
Posted by bo...@apache.org.
Merge branch 'master' into ClusterUtils
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/cc1f6d77
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/cc1f6d77
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/cc1f6d77
Branch: refs/heads/master
Commit: cc1f6d77962ec96c69cf5af85e4016a9e08ccdcc
Parents: afcd0c6 d0cd52b
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Fri Feb 19 16:22:50 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Fri Feb 19 16:47:42 2016 +0800
----------------------------------------------------------------------
CHANGELOG.md | 7 +
README.markdown | 1 +
bin/storm.cmd | 2 +-
bin/storm.py | 4 +-
conf/cgconfig.conf.example | 41 +++
conf/defaults.yaml | 16 +-
examples/storm-starter/pom.xml | 10 +
.../org/apache/storm/starter/clj/word_count.clj | 3 +-
.../starter/ResourceAwareExampleTopology.java | 2 +-
pom.xml | 10 +
storm-clojure/pom.xml | 74 ++++
.../src/clj/org/apache/storm/clojure.clj | 207 +++++++++++
.../src/clj/org/apache/storm/thrift.clj | 286 +++++++++++++++
storm-clojure/src/test/clj/clojure_test.clj | 158 +++++++++
storm-core/src/clj/org/apache/storm/clojure.clj | 207 -----------
.../org/apache/storm/command/dev_zookeeper.clj | 28 --
.../clj/org/apache/storm/command/get_errors.clj | 3 +-
.../org/apache/storm/command/healthcheck.clj | 90 -----
.../clj/org/apache/storm/command/monitor.clj | 2 +-
.../clj/org/apache/storm/command/rebalance.clj | 3 +-
.../org/apache/storm/command/set_log_level.clj | 3 +-
.../apache/storm/command/shell_submission.clj | 2 +-
.../src/clj/org/apache/storm/daemon/common.clj | 119 ++++---
.../clj/org/apache/storm/daemon/executor.clj | 51 +--
.../clj/org/apache/storm/daemon/logviewer.clj | 19 +-
.../src/clj/org/apache/storm/daemon/nimbus.clj | 122 +++----
.../clj/org/apache/storm/daemon/supervisor.clj | 217 ++++++++----
.../src/clj/org/apache/storm/daemon/task.clj | 4 +-
.../src/clj/org/apache/storm/daemon/worker.clj | 116 +++---
.../clj/org/apache/storm/internal/clojure.clj | 201 +++++++++++
.../clj/org/apache/storm/internal/thrift.clj | 96 +++++
.../src/clj/org/apache/storm/local_state.clj | 134 -------
.../org/apache/storm/local_state_converter.clj | 24 ++
storm-core/src/clj/org/apache/storm/testing.clj | 37 +-
storm-core/src/clj/org/apache/storm/thrift.clj | 286 ---------------
storm-core/src/clj/org/apache/storm/timer.clj | 128 -------
storm-core/src/clj/org/apache/storm/ui/core.clj | 2 +-
storm-core/src/jvm/org/apache/storm/Config.java | 88 +++++
.../src/jvm/org/apache/storm/StormTimer.java | 241 +++++++++++++
storm-core/src/jvm/org/apache/storm/Thrift.java | 351 +++++++++++++++++++
.../org/apache/storm/command/DevZookeeper.java | 35 ++
.../org/apache/storm/command/HealthCheck.java | 125 +++++++
.../container/ResourceIsolationInterface.java | 51 +++
.../storm/container/cgroup/CgroupCenter.java | 216 ++++++++++++
.../storm/container/cgroup/CgroupCommon.java | 270 ++++++++++++++
.../container/cgroup/CgroupCommonOperation.java | 81 +++++
.../container/cgroup/CgroupCoreFactory.java | 74 ++++
.../storm/container/cgroup/CgroupManager.java | 210 +++++++++++
.../storm/container/cgroup/CgroupOperation.java | 79 +++++
.../storm/container/cgroup/CgroupUtils.java | 118 +++++++
.../apache/storm/container/cgroup/Device.java | 75 ++++
.../storm/container/cgroup/Hierarchy.java | 130 +++++++
.../storm/container/cgroup/SubSystem.java | 81 +++++
.../storm/container/cgroup/SubSystemType.java | 36 ++
.../storm/container/cgroup/SystemOperation.java | 75 ++++
.../storm/container/cgroup/core/BlkioCore.java | 213 +++++++++++
.../storm/container/cgroup/core/CgroupCore.java | 26 ++
.../storm/container/cgroup/core/CpuCore.java | 135 +++++++
.../container/cgroup/core/CpuacctCore.java | 71 ++++
.../storm/container/cgroup/core/CpusetCore.java | 209 +++++++++++
.../container/cgroup/core/DevicesCore.java | 189 ++++++++++
.../container/cgroup/core/FreezerCore.java | 66 ++++
.../storm/container/cgroup/core/MemoryCore.java | 188 ++++++++++
.../storm/container/cgroup/core/NetClsCore.java | 69 ++++
.../container/cgroup/core/NetPrioCore.java | 65 ++++
.../jvm/org/apache/storm/testing/NGrouping.java | 4 +-
.../storm/testing/PythonShellMetricsBolt.java | 14 +-
.../storm/testing/PythonShellMetricsSpout.java | 8 +-
.../jvm/org/apache/storm/utils/LocalState.java | 112 +++++-
.../src/jvm/org/apache/storm/utils/Utils.java | 17 +-
.../org/apache/storm/integration_test.clj | 259 +++++++-------
.../org/apache/storm/testing4j_test.clj | 72 ++--
.../test/clj/org/apache/storm/clojure_test.clj | 64 ++--
.../test/clj/org/apache/storm/cluster_test.clj | 3 +-
.../test/clj/org/apache/storm/drpc_test.clj | 23 +-
.../test/clj/org/apache/storm/grouping_test.clj | 56 +--
.../storm/messaging/netty_integration_test.clj | 19 +-
.../clj/org/apache/storm/messaging_test.clj | 14 +-
.../test/clj/org/apache/storm/metrics_test.clj | 85 +++--
.../test/clj/org/apache/storm/nimbus_test.clj | 259 +++++++++-----
.../scheduler/resource_aware_scheduler_test.clj | 3 +-
.../clj/org/apache/storm/supervisor_test.clj | 55 +--
.../clj/org/apache/storm/tick_tuple_test.clj | 15 +-
.../clj/org/apache/storm/transactional_test.clj | 3 +-
.../test/jvm/org/apache/storm/TestCgroups.java | 130 +++++++
.../resource/TestResourceAwareScheduler.java | 3 +
86 files changed, 5939 insertions(+), 1561 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/conf/defaults.yaml
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-clojure/src/clj/org/apache/storm/thrift.clj
----------------------------------------------------------------------
diff --cc storm-clojure/src/clj/org/apache/storm/thrift.clj
index 0000000,bf13d23..45aa740
mode 000000,100644..100644
--- a/storm-clojure/src/clj/org/apache/storm/thrift.clj
+++ b/storm-clojure/src/clj/org/apache/storm/thrift.clj
@@@ -1,0 -1,286 +1,286 @@@
+ ;; 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 org.apache.storm.thrift
+ (:import [java.util HashMap]
+ [java.io Serializable]
+ [org.apache.storm.generated NodeInfo Assignment])
+ (:import [org.apache.storm.generated JavaObject Grouping Nimbus StormTopology
+ StormTopology$_Fields Bolt Nimbus$Client Nimbus$Iface
+ ComponentCommon Grouping$_Fields SpoutSpec NullStruct StreamInfo
+ GlobalStreamId ComponentObject ComponentObject$_Fields
+ ShellComponent SupervisorInfo])
+ (:import [org.apache.storm.utils Utils NimbusClient ConfigUtils])
+ (:import [org.apache.storm Constants])
+ (:import [org.apache.storm.security.auth ReqContext])
+ (:import [org.apache.storm.grouping CustomStreamGrouping])
+ (:import [org.apache.storm.topology TopologyBuilder])
+ (:import [org.apache.storm.clojure RichShellBolt RichShellSpout])
+ (:import [org.apache.storm.thrift.transport TTransport]
+ (org.json.simple JSONValue))
- (:use [org.apache.storm util config log zookeeper]))
++ (:use [org.apache.storm util config log]))
+
+ (defn instantiate-java-object
+ [^JavaObject obj]
+ (let [name (symbol (.get_full_class_name obj))
+ args (map (memfn getFieldValue) (.get_args_list obj))]
+ (eval `(new ~name ~@args))))
+
+ (def grouping-constants
+ {Grouping$_Fields/FIELDS :fields
+ Grouping$_Fields/SHUFFLE :shuffle
+ Grouping$_Fields/ALL :all
+ Grouping$_Fields/NONE :none
+ Grouping$_Fields/CUSTOM_SERIALIZED :custom-serialized
+ Grouping$_Fields/CUSTOM_OBJECT :custom-object
+ Grouping$_Fields/DIRECT :direct
+ Grouping$_Fields/LOCAL_OR_SHUFFLE :local-or-shuffle})
+
+ (defn grouping-type
+ [^Grouping grouping]
+ (grouping-constants (.getSetField grouping)))
+
+ (defn field-grouping
+ [^Grouping grouping]
+ (when-not (= (grouping-type grouping) :fields)
+ (throw (IllegalArgumentException. "Tried to get grouping fields from non fields grouping")))
+ (.get_fields grouping))
+
+ (defn global-grouping?
+ [^Grouping grouping]
+ (and (= :fields (grouping-type grouping))
+ (empty? (field-grouping grouping))))
+
+ (defn parallelism-hint
+ [^ComponentCommon component-common]
+ (let [phint (.get_parallelism_hint component-common)]
+ (if-not (.is_set_parallelism_hint component-common) 1 phint)))
+
+ (defn nimbus-client-and-conn
+ ([host port]
+ (nimbus-client-and-conn host port nil))
+ ([host port as-user]
+ (log-message "Connecting to Nimbus at " host ":" port " as user: " as-user)
+ (let [conf (clojurify-structure (ConfigUtils/readStormConfig))
+ nimbusClient (NimbusClient. conf host port nil as-user)
+ client (.getClient nimbusClient)
+ transport (.transport nimbusClient)]
+ [client transport] )))
+
+ (defmacro with-nimbus-connection
+ [[client-sym host port] & body]
+ `(let [[^Nimbus$Client ~client-sym ^TTransport conn#] (nimbus-client-and-conn ~host ~port)]
+ (try
+ ~@body
+ (finally (.close conn#)))))
+
+ (defmacro with-configured-nimbus-connection
+ [client-sym & body]
+ `(let [conf# (clojurify-structure (ConfigUtils/readStormConfig))
+ context# (ReqContext/context)
+ user# (if (.principal context#) (.getName (.principal context#)))
+ nimbusClient# (NimbusClient/getConfiguredClientAs conf# user#)
+ ~client-sym (.getClient nimbusClient#)
+ conn# (.transport nimbusClient#)
+ ]
+ (try
+ ~@body
+ (finally (.close conn#)))))
+
+ (defn direct-output-fields
+ [fields]
+ (StreamInfo. fields true))
+
+ (defn output-fields
+ [fields]
+ (StreamInfo. fields false))
+
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
+ (defn mk-output-spec
+ [output-spec]
+ (let [output-spec (if (map? output-spec)
+ output-spec
+ {Utils/DEFAULT_STREAM_ID output-spec})]
+ (map-val
+ (fn [out]
+ (if (instance? StreamInfo out)
+ out
+ (StreamInfo. out false)))
+ output-spec)))
+
+ (defnk mk-plain-component-common
+ [inputs output-spec parallelism-hint :conf nil]
+ (let [ret (ComponentCommon. (HashMap. inputs) (HashMap. (mk-output-spec output-spec)))]
+ (when parallelism-hint
+ (.set_parallelism_hint ret parallelism-hint))
+ (when conf
+ (.set_json_conf ret (JSONValue/toJSONString conf)))
+ ret))
+
+ (defnk mk-spout-spec*
+ [spout outputs :p nil :conf nil]
+ (SpoutSpec. (ComponentObject/serialized_java (Utils/javaSerialize spout))
+ (mk-plain-component-common {} outputs p :conf conf)))
+
+ (defn mk-shuffle-grouping
+ []
+ (Grouping/shuffle (NullStruct.)))
+
+ (defn mk-local-or-shuffle-grouping
+ []
+ (Grouping/local_or_shuffle (NullStruct.)))
+
+ (defn mk-fields-grouping
+ [fields]
+ (Grouping/fields fields))
+
+ (defn mk-global-grouping
+ []
+ (mk-fields-grouping []))
+
+ (defn mk-direct-grouping
+ []
+ (Grouping/direct (NullStruct.)))
+
+ (defn mk-all-grouping
+ []
+ (Grouping/all (NullStruct.)))
+
+ (defn mk-none-grouping
+ []
+ (Grouping/none (NullStruct.)))
+
+ (defn deserialized-component-object
+ [^ComponentObject obj]
+ (when (not= (.getSetField obj) ComponentObject$_Fields/SERIALIZED_JAVA)
+ (throw (RuntimeException. "Cannot deserialize non-java-serialized object")))
+ (Utils/javaDeserialize (.get_serialized_java obj) Serializable))
+
+ (defn serialize-component-object
+ [obj]
+ (ComponentObject/serialized_java (Utils/javaSerialize obj)))
+
+ (defn- mk-grouping
+ [grouping-spec]
+ (cond (nil? grouping-spec)
+ (mk-none-grouping)
+
+ (instance? Grouping grouping-spec)
+ grouping-spec
+
+ (instance? CustomStreamGrouping grouping-spec)
+ (Grouping/custom_serialized (Utils/javaSerialize grouping-spec))
+
+ (instance? JavaObject grouping-spec)
+ (Grouping/custom_object grouping-spec)
+
+ (sequential? grouping-spec)
+ (mk-fields-grouping grouping-spec)
+
+ (= grouping-spec :shuffle)
+ (mk-shuffle-grouping)
+
+ (= grouping-spec :local-or-shuffle)
+ (mk-local-or-shuffle-grouping)
+ (= grouping-spec :none)
+ (mk-none-grouping)
+
+ (= grouping-spec :all)
+ (mk-all-grouping)
+
+ (= grouping-spec :global)
+ (mk-global-grouping)
+
+ (= grouping-spec :direct)
+ (mk-direct-grouping)
+
+ true
+ (throw (IllegalArgumentException.
+ (str grouping-spec " is not a valid grouping")))))
+
+ (defn- mk-inputs
+ [inputs]
+ (into {} (for [[stream-id grouping-spec] inputs]
+ [(if (sequential? stream-id)
+ (GlobalStreamId. (first stream-id) (second stream-id))
+ (GlobalStreamId. stream-id Utils/DEFAULT_STREAM_ID))
+ (mk-grouping grouping-spec)])))
+
+ (defnk mk-bolt-spec*
+ [inputs bolt outputs :p nil :conf nil]
+ (let [common (mk-plain-component-common (mk-inputs inputs) outputs p :conf conf)]
+ (Bolt. (ComponentObject/serialized_java (Utils/javaSerialize bolt))
+ common)))
+
+ (defnk mk-spout-spec
+ [spout :parallelism-hint nil :p nil :conf nil]
+ (let [parallelism-hint (if p p parallelism-hint)]
+ {:obj spout :p parallelism-hint :conf conf}))
+
+ (defn- shell-component-params
+ [command script-or-output-spec kwargs]
+ (if (string? script-or-output-spec)
+ [(into-array String [command script-or-output-spec])
+ (first kwargs)
+ (rest kwargs)]
+ [(into-array String command)
+ script-or-output-spec
+ kwargs]))
+
+ (defnk mk-bolt-spec
+ [inputs bolt :parallelism-hint nil :p nil :conf nil]
+ (let [parallelism-hint (if p p parallelism-hint)]
+ {:obj bolt :inputs inputs :p parallelism-hint :conf conf}))
+
+ (defn mk-shell-bolt-spec
+ [inputs command script-or-output-spec & kwargs]
+ (let [[command output-spec kwargs]
+ (shell-component-params command script-or-output-spec kwargs)]
+ (apply mk-bolt-spec inputs
+ (RichShellBolt. command (mk-output-spec output-spec)) kwargs)))
+
+ (defn mk-shell-spout-spec
+ [command script-or-output-spec & kwargs]
+ (let [[command output-spec kwargs]
+ (shell-component-params command script-or-output-spec kwargs)]
+ (apply mk-spout-spec
+ (RichShellSpout. command (mk-output-spec output-spec)) kwargs)))
+
+ (defn- add-inputs
+ [declarer inputs]
+ (doseq [[id grouping] (mk-inputs inputs)]
+ (.grouping declarer id grouping)))
+
+ (defn mk-topology
+ ([spout-map bolt-map]
+ (let [builder (TopologyBuilder.)]
+ (doseq [[name {spout :obj p :p conf :conf}] spout-map]
+ (-> builder (.setSpout name spout (if-not (nil? p) (int p) p)) (.addConfigurations conf)))
+ (doseq [[name {bolt :obj p :p conf :conf inputs :inputs}] bolt-map]
+ (-> builder (.setBolt name bolt (if-not (nil? p) (int p) p)) (.addConfigurations conf) (add-inputs inputs)))
+ (.createTopology builder)))
+ ([spout-map bolt-map state-spout-map]
+ (mk-topology spout-map bolt-map)))
+
+ ;; clojurify-structure is needed or else every element becomes the same after successive calls
+ ;; don't know why this happens
+ (def STORM-TOPOLOGY-FIELDS
+ (-> StormTopology/metaDataMap clojurify-structure keys))
+
+ (def SPOUT-FIELDS
+ [StormTopology$_Fields/SPOUTS
+ StormTopology$_Fields/STATE_SPOUTS])
+
http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/command/shell_submission.clj
index 870e7f6,0d29376..8d193e6
--- a/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
+++ b/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
@@@ -17,7 -17,7 +17,7 @@@
(:import [org.apache.storm StormSubmitter]
[org.apache.storm.utils Utils]
[org.apache.storm.zookeeper Zookeeper])
- (:use [org.apache.storm thrift util config log])
- (:use [org.apache.storm util config log zookeeper])
++ (:use [org.apache.storm util config log])
(:require [clojure.string :as str])
(:import [org.apache.storm.utils ConfigUtils])
(:gen-class))
http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/src/clj/org/apache/storm/daemon/common.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/common.clj
index db342d2,db7fd40..99b34dc
--- a/storm-core/src/clj/org/apache/storm/daemon/common.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/common.clj
@@@ -15,8 -15,8 +15,8 @@@
;; limitations under the License.
(ns org.apache.storm.daemon.common
(:use [org.apache.storm log config util])
- (:import [org.apache.storm.generated StormTopology
+ (:import [org.apache.storm.generated StormTopology NodeInfo
- InvalidTopologyException GlobalStreamId]
+ InvalidTopologyException GlobalStreamId Grouping Grouping$_Fields]
[org.apache.storm.utils Utils ConfigUtils IPredicate ThriftTopologyUtils]
[org.apache.storm.daemon.metrics.reporters PreparableReporter]
[com.codahale.metrics MetricRegistry])
http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/executor.clj
index 902650c,92cc003..9ff93f8
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@@ -39,8 -39,9 +39,9 @@@
(:import [java.lang Thread Thread$UncaughtExceptionHandler]
[java.util.concurrent ConcurrentLinkedQueue]
[org.json.simple JSONValue]
- [com.lmax.disruptor.dsl ProducerType])
- (:require [org.apache.storm [thrift :as thrift] [stats :as stats]])
+ [com.lmax.disruptor.dsl ProducerType]
+ [org.apache.storm StormTimer])
- (:require [org.apache.storm [cluster :as cluster] [stats :as stats]])
++ (:require [org.apache.storm [stats :as stats]])
(:require [org.apache.storm.daemon [task :as task]])
(:require [org.apache.storm.daemon.builtin-metrics :as builtin-metrics])
(:require [clojure.set :as set]))
http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index beb6639,28a6fb8..7d28075
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@@ -46,13 -46,14 +46,13 @@@
KillOptions RebalanceOptions ClusterSummary SupervisorSummary TopologySummary TopologyInfo TopologyHistoryInfo
ExecutorSummary AuthorizationException GetInfoOptions NumErrorsChoice SettableBlobMeta ReadableBlobMeta
BeginDownloadResult ListBlobsResult ComponentPageInfo TopologyPageInfo LogConfig LogLevel LogLevelAction
- ProfileRequest ProfileAction NodeInfo])
+ ProfileRequest ProfileAction NodeInfo LSTopoHistory])
(:import [org.apache.storm.daemon Shutdownable])
(:import [org.apache.storm.validation ConfigValidation])
- (:import [org.apache.storm.cluster ClusterStateContext DaemonType])
- (:use [org.apache.storm util config log zookeeper])
- (:require [org.apache.storm [cluster :as cluster]
- [converter :as converter]
- [stats :as stats]])
+ (:import [org.apache.storm.cluster ClusterStateContext DaemonType StormClusterStateImpl ClusterUtils])
- (:use [org.apache.storm util config log timer local-state converter])
++ (:use [org.apache.storm util config log converter])
+ (:require [org.apache.storm [converter :as converter]
- [stats :as stats]])
++ [stats :as stats]])
(:require [clojure.set :as set])
(:import [org.apache.storm.daemon.common StormBase Assignment])
(:import [org.apache.storm.zookeeper Zookeeper])
@@@ -270,17 -279,17 +274,17 @@@
{:status {:type :rebalancing}
:prev-status status
:topology-action-options (-> {:delay-secs delay :action :rebalance}
-- (assoc-non-nil :num-workers num-workers)
-- (assoc-non-nil :component->executors executor-overrides))
++ (converter/assoc-non-nil :num-workers num-workers)
++ (converter/assoc-non-nil :component->executors executor-overrides))
})))
(defn do-rebalance [nimbus storm-id status storm-base]
(let [rebalance-options (:topology-action-options storm-base)]
- (.update-storm! (:storm-cluster-state nimbus)
+ (.updateStorm (:storm-cluster-state nimbus)
storm-id
- (thriftify-storm-base (-> {:topology-action-options nil}
- (-> {:topology-action-options nil}
-- (assoc-non-nil :component->executors (:component->executors rebalance-options))
- (assoc-non-nil :num-workers (:num-workers rebalance-options))))))
- (assoc-non-nil :num-workers (:num-workers rebalance-options)))))
++ (converter/thriftify-storm-base (-> {:topology-action-options nil}
++ (converter/assoc-non-nil :component->executors (:component->executors rebalance-options))
++ (converter/assoc-non-nil :num-workers (:num-workers rebalance-options))))))
(mk-assignments nimbus :scratch-topology-id storm-id))
(defn state-transitions [nimbus storm-id status storm-base]
@@@ -363,7 -372,7 +367,7 @@@
storm-base-updates)]
(when storm-base-updates
- (.updateStorm (:storm-cluster-state nimbus) storm-id (thriftify-storm-base storm-base-updates))))))
- (.update-storm! (:storm-cluster-state nimbus) storm-id storm-base-updates)))))
++ (.updateStorm (:storm-cluster-state nimbus) storm-id (converter/thriftify-storm-base storm-base-updates))))))
)))
(defn transition-name! [nimbus storm-name event & args]
@@@ -1002,9 -1005,9 +1005,9 @@@
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
+ (.activateStorm storm-cluster-state
storm-id
- (thriftify-storm-base (StormBase. storm-name
- (StormBase. storm-name
++ (converter/thriftify-storm-base (StormBase. storm-name
(Time/currentTimeSecs)
{:type topology-initial-status}
(storm-conf TOPOLOGY-WORKERS)
@@@ -1448,41 -1446,39 +1454,39 @@@
(setup-blobstore nimbus))
(when (is-leader nimbus :throw-exception false)
- (doseq [storm-id (.active-storms (:storm-cluster-state nimbus))]
+ (doseq [storm-id (.activeStorms (:storm-cluster-state nimbus))]
(transition! nimbus storm-id :startup)))
- (schedule-recurring (:timer nimbus)
- 0
- (conf NIMBUS-MONITOR-FREQ-SECS)
- (fn []
- (when-not (conf ConfigUtils/NIMBUS_DO_NOT_REASSIGN)
- (locking (:submit-lock nimbus)
- (mk-assignments nimbus)))
- (do-cleanup nimbus)))
+
+ (.scheduleRecurring (:timer nimbus)
+ 0
+ (conf NIMBUS-MONITOR-FREQ-SECS)
+ (fn []
+ (when-not (conf ConfigUtils/NIMBUS_DO_NOT_REASSIGN)
+ (locking (:submit-lock nimbus)
+ (mk-assignments 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))))
+ (.scheduleRecurring (:timer nimbus)
+ 0
+ (conf NIMBUS-CLEANUP-INBOX-FREQ-SECS)
+ (fn [] (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))))
+ (.scheduleRecurring (: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)
+ (.scheduleRecurring (:timer nimbus)
0
(conf LOGVIEWER-CLEANUP-INTERVAL-SECS)
- (fn []
- (clean-topology-history (conf LOGVIEWER-CLEANUP-AGE-MINS) nimbus))))
- (schedule-recurring (:timer nimbus)
- 0
- (conf NIMBUS-CREDENTIAL-RENEW-FREQ-SECS)
- (fn []
- (renew-credentials nimbus)))
+ (fn [] (clean-topology-history (conf LOGVIEWER-CLEANUP-AGE-MINS) nimbus))))
+ (.scheduleRecurring (:timer nimbus)
+ 0
+ (conf NIMBUS-CREDENTIAL-RENEW-FREQ-SECS)
+ (fn []
+ (renew-credentials nimbus)))
(defgauge nimbus:num-supervisors
(fn [] (.size (.supervisors (:storm-cluster-state nimbus) nil))))
@@@ -1650,7 -1646,7 +1654,7 @@@
(log-message "Nimbus setting debug to " enable? " for storm-name '" storm-name "' storm-id '" storm-id "' sampling pct '" spct "'"
(if (not (clojure.string/blank? component-id)) (str " component-id '" component-id "'")))
(locking (:submit-lock nimbus)
- (.updateStorm storm-cluster-state storm-id (thriftify-storm-base storm-base-updates)))))
- (.update-storm! storm-cluster-state storm-id storm-base-updates))))
++ (.updateStorm storm-cluster-state storm-id (converter/thriftify-storm-base storm-base-updates)))))
(^void setWorkerProfiler
[this ^String id ^ProfileRequest profileRequest]
http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
index 58f6291,fad5b1a..f429d09
--- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
@@@ -24,17 -24,17 +24,17 @@@
[java.net JarURLConnection]
[java.net URI URLDecoder]
[org.apache.commons.io FileUtils])
- (:use [org.apache.storm config util log timer local-state converter])
- (:use [org.apache.storm config util log local-state-converter])
++ (:use [org.apache.storm config util log converter local-state-converter])
(:import [org.apache.storm.generated AuthorizationException KeyNotFoundException WorkerResources])
(:import [org.apache.storm.utils NimbusLeaderNotFoundException VersionInfo])
(:import [java.nio.file Files StandardCopyOption])
(:import [org.apache.storm Config])
- (:import [org.apache.storm.generated WorkerResources ProfileAction])
+ (:import [org.apache.storm.generated WorkerResources ProfileAction LocalAssignment])
(:import [org.apache.storm.localizer LocalResource])
(:use [org.apache.storm.daemon common])
- (:require [org.apache.storm.command [healthcheck :as healthcheck]])
+ (:import [org.apache.storm.command HealthCheck])
(:require [org.apache.storm.daemon [worker :as worker]]
- [org.apache.storm [process-simulator :as psim] [cluster :as cluster] [event :as event]]
+ [org.apache.storm [process-simulator :as psim] [event :as event]]
[clojure.set :as set])
(:import [org.apache.thrift.transport TTransportException])
(:import [org.apache.zookeeper data.ACL ZooDefs$Ids ZooDefs$Perms])
http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/src/clj/org/apache/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/worker.clj
index 395be23,8f9becd..a75dc35
--- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
@@@ -15,7 -15,7 +15,7 @@@
;; limitations under the License.
(ns org.apache.storm.daemon.worker
(:use [org.apache.storm.daemon common])
- (:use [org.apache.storm config log util timer local-state converter])
- (:use [org.apache.storm config log util local-state-converter])
++ (:use [org.apache.storm config log util converter local-state-converter])
(:require [clj-time.core :as time])
(:require [clj-time.coerce :as coerce])
(:require [org.apache.storm.daemon [executor :as executor]])
@@@ -238,13 -243,14 +243,14 @@@
{})
(defn mk-halting-timer [timer-name]
- (mk-timer :kill-fn (fn [t]
- (log-error t "Error when processing event")
- (Utils/exitProcess 20 "Error when processing an event")
- )
- :timer-name timer-name))
+ (StormTimer. timer-name
+ (reify Thread$UncaughtExceptionHandler
+ (^void uncaughtException
+ [this ^Thread t ^Throwable e]
+ (log-error e "Error when processing event")
+ (Utils/exitProcess 20 "Error when processing an event")))))
-(defn worker-data [conf mq-context storm-id assignment-id port worker-id storm-conf cluster-state storm-cluster-state]
+(defn worker-data [conf mq-context storm-id assignment-id port worker-id storm-conf state-store storm-cluster-state]
(let [assignment-versions (atom {})
executors (set (read-worker-executors storm-conf storm-cluster-state storm-id assignment-id port assignment-versions))
transfer-queue (DisruptorQueue. "worker-transfer-queue"
@@@ -374,15 -380,16 +380,17 @@@
conf (:conf worker)
storm-cluster-state (:storm-cluster-state worker)
storm-id (:storm-id worker)]
- (fn this
+ (fn refresh-connections
([]
- (this (fn [& ignored] (schedule (:refresh-connections-timer worker) 0 this))))
+ (refresh-connections (fn [& ignored]
+ (.schedule
+ (:refresh-connections-timer worker) 0 refresh-connections))))
([callback]
- (let [version (.assignment-version storm-cluster-state storm-id callback)
+ (let [version (.assignmentVersion storm-cluster-state storm-id callback)
assignment (if (= version (:version (get @(:assignment-versions worker) storm-id)))
(:data (get @(:assignment-versions worker) storm-id))
- (let [new-assignment (.assignment-info-with-version storm-cluster-state storm-id callback)]
+ (let [thriftify-assignment-version (.assignmentInfoWithVersion storm-cluster-state storm-id callback)
+ new-assignment {:data (clojurify-assignment (.get thriftify-assignment-version (IStateStorage/DATA))) :version version}]
(swap! (:assignment-versions worker) assoc storm-id new-assignment)
(:data new-assignment)))
my-assignment (-> assignment
@@@ -428,9 -435,12 +436,12 @@@
(defn refresh-storm-active
([worker]
- (refresh-storm-active worker (fn [& ignored] (schedule (:refresh-active-timer worker) 0 (partial refresh-storm-active worker)))))
+ (refresh-storm-active
+ worker (fn [& ignored]
+ (.schedule
+ (:refresh-active-timer worker) 0 (partial refresh-storm-active worker)))))
([worker callback]
- (let [base (.storm-base (:storm-cluster-state worker) (:storm-id worker) callback)]
+ (let [base (clojurify-storm-base (.stormBase (:storm-cluster-state worker) (:storm-id worker) callback))]
(reset!
(:storm-active-atom worker)
(and (= :active (-> base :status :type)) @(:worker-active-flag worker)))
@@@ -757,22 -768,28 +769,28 @@@
(log-message "Started with log levels: " @original-log-levels)
(defn establish-log-setting-callback []
- (.topology-log-config (:storm-cluster-state worker) storm-id (fn [args] (check-log-config-changed))))
+ (.topologyLogConfig (:storm-cluster-state worker) storm-id (fn [args] (check-log-config-changed))))
(establish-log-setting-callback)
- (.credentials (:storm-cluster-state worker) storm-id (fn [args] (check-credentials-changed)))
+ (clojurify-crdentials (.credentials (:storm-cluster-state worker) storm-id (fn [] (check-credentials-changed))))
- (schedule-recurring (:refresh-credentials-timer worker) 0 (conf TASK-CREDENTIALS-POLL-SECS)
- (fn [& args]
- (check-credentials-changed)
- (if ((:storm-conf worker) TOPOLOGY-BACKPRESSURE-ENABLE)
- (check-throttle-changed))))
+
+ (.scheduleRecurring
+ (:refresh-credentials-timer worker) 0 (conf TASK-CREDENTIALS-POLL-SECS)
+ (fn []
+ (check-credentials-changed)
+ (if ((:storm-conf worker) TOPOLOGY-BACKPRESSURE-ENABLE)
+ (check-throttle-changed))))
;; The jitter allows the clients to get the data at different times, and avoids thundering herd
(when-not (.get conf TOPOLOGY-DISABLE-LOADAWARE-MESSAGING)
- (schedule-recurring-with-jitter (:refresh-load-timer worker) 0 1 500 refresh-load))
- (schedule-recurring (:refresh-connections-timer worker) 0 (conf TASK-REFRESH-POLL-SECS) refresh-connections)
- (schedule-recurring (:reset-log-levels-timer worker) 0 (conf WORKER-LOG-LEVEL-RESET-POLL-SECS) (fn [] (reset-log-levels latest-log-config)))
- (schedule-recurring (:refresh-active-timer worker) 0 (conf TASK-REFRESH-POLL-SECS) (partial refresh-storm-active worker))
-
+ (.scheduleRecurringWithJitter
+ (:refresh-load-timer worker) 0 1 500 refresh-load))
+ (.scheduleRecurring
+ (:refresh-connections-timer worker) 0 (conf TASK-REFRESH-POLL-SECS) refresh-connections)
+ (.scheduleRecurring
+ (:reset-log-levels-timer worker) 0 (conf WORKER-LOG-LEVEL-RESET-POLL-SECS)
+ (fn [] (reset-log-levels latest-log-config)))
+ (.scheduleRecurring
+ (:refresh-active-timer worker) 0 (conf TASK-REFRESH-POLL-SECS) (partial refresh-storm-active worker))
(log-message "Worker has topology config " (Utils/redactValue (:storm-conf worker) STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD))
(log-message "Worker " worker-id " for storm " storm-id " on " assignment-id ":" port " has finished loading")
ret
http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/src/clj/org/apache/storm/internal/thrift.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/internal/thrift.clj
index 0000000,4ccf8a7..07e0bdf
mode 000000,100644..100644
--- a/storm-core/src/clj/org/apache/storm/internal/thrift.clj
+++ b/storm-core/src/clj/org/apache/storm/internal/thrift.clj
@@@ -1,0 -1,96 +1,96 @@@
+ ;; 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 org.apache.storm.internal.thrift
+ (:import [java.util HashMap]
+ [java.io Serializable]
+ [org.apache.storm.generated NodeInfo Assignment])
+ (:import [org.apache.storm.generated JavaObject Grouping Nimbus StormTopology
+ StormTopology$_Fields Bolt Nimbus$Client Nimbus$Iface
+ ComponentCommon Grouping$_Fields SpoutSpec NullStruct StreamInfo
+ GlobalStreamId ComponentObject ComponentObject$_Fields
+ ShellComponent SupervisorInfo])
+ (:import [org.apache.storm.utils Utils NimbusClient ConfigUtils])
+ (:import [org.apache.storm Constants])
+ (:import [org.apache.storm.security.auth ReqContext])
+ (:import [org.apache.storm.grouping CustomStreamGrouping])
+ (:import [org.apache.storm.topology TopologyBuilder])
+ (:import [org.apache.storm.clojure RichShellBolt RichShellSpout])
+ (:import [org.apache.thrift.transport TTransport])
- (:use [org.apache.storm util config log zookeeper]))
++ (:use [org.apache.storm util config log]))
+
+ ;; Leaving this definition as core.clj is using them as a nested keyword argument
+ ;; Must remove once core.clj is ported to java
+ (def grouping-constants
+ {Grouping$_Fields/FIELDS :fields
+ Grouping$_Fields/SHUFFLE :shuffle
+ Grouping$_Fields/ALL :all
+ Grouping$_Fields/NONE :none
+ Grouping$_Fields/CUSTOM_SERIALIZED :custom-serialized
+ Grouping$_Fields/CUSTOM_OBJECT :custom-object
+ Grouping$_Fields/DIRECT :direct
+ Grouping$_Fields/LOCAL_OR_SHUFFLE :local-or-shuffle})
+
+ ;; Leaving this method as core.clj is using them as a nested keyword argument
+ ;; Must remove once core.clj is ported to java
+ (defn grouping-type
+ [^Grouping grouping]
+ (grouping-constants (.getSetField grouping)))
+
+ (defn nimbus-client-and-conn
+ ([host port]
+ (nimbus-client-and-conn host port nil))
+ ([host port as-user]
+ (log-message "Connecting to Nimbus at " host ":" port " as user: " as-user)
+ (let [conf (clojurify-structure (ConfigUtils/readStormConfig))
+ nimbusClient (NimbusClient. conf host port nil as-user)
+ client (.getClient nimbusClient)
+ transport (.transport nimbusClient)]
+ [client transport] )))
+
+ (defmacro with-nimbus-connection
+ [[client-sym host port] & body]
+ `(let [[^Nimbus$Client ~client-sym ^TTransport conn#] (nimbus-client-and-conn ~host ~port)]
+ (try
+ ~@body
+ (finally (.close conn#)))))
+
+ (defmacro with-configured-nimbus-connection
+ [client-sym & body]
+ `(let [conf# (clojurify-structure (ConfigUtils/readStormConfig))
+ context# (ReqContext/context)
+ user# (if (.principal context#) (.getName (.principal context#)))
+ nimbusClient# (NimbusClient/getConfiguredClientAs conf# user#)
+ ~client-sym (.getClient nimbusClient#)
+ conn# (.transport nimbusClient#)
+ ]
+ (try
+ ~@body
+ (finally (.close conn#)))))
+
+ ;; Leaving this definition as core.clj is using them as a nested keyword argument
+ ;; Must remove once core.clj is ported to java
+ (defn mk-output-spec
+ [output-spec]
+ (let [output-spec (if (map? output-spec)
+ output-spec
+ {Utils/DEFAULT_STREAM_ID output-spec})]
+ (map-val
+ (fn [out]
+ (if (instance? StreamInfo out)
+ out
+ (StreamInfo. out false)))
+ output-spec)))
http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/src/clj/org/apache/storm/testing.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/testing.clj
index 3dee54b,7817929..8f313cc
--- a/storm-core/src/clj/org/apache/storm/testing.clj
+++ b/storm-core/src/clj/org/apache/storm/testing.clj
@@@ -48,9 -49,10 +49,10 @@@
(:import [org.apache.storm.task TopologyContext]
(org.apache.storm.messaging IContext)
[org.json.simple JSONValue])
- (:require [org.apache.storm [zookeeper :as zk]])
+ (:import [org.apache.storm.cluster ZKStateStorage ClusterStateContext StormClusterStateImpl ClusterUtils])
(:require [org.apache.storm.daemon.acker :as acker])
- (:use [org.apache.storm util thrift config log local-state converter]))
- (:use [org.apache.storm cluster util config log local-state-converter])
++ (:use [org.apache.storm util config log local-state-converter converter])
+ (:use [org.apache.storm.internal thrift]))
(defn feeder-spout
[fields]
http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/src/clj/org/apache/storm/ui/core.clj
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/integration/org/apache/storm/integration_test.clj
index 1bb8279,6dce7d6..697bdae
--- a/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
+++ b/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
@@@ -21,10 -21,11 +21,12 @@@
(:import [org.apache.storm.testing TestWordCounter TestWordSpout TestGlobalCount
TestAggregatesCounter TestConfBolt AckFailMapTracker AckTracker TestPlannerSpout])
(:import [org.apache.storm.tuple Fields])
- (:use [org.apache.storm testing config util])
+ (:import [org.apache.storm.cluster StormClusterStateImpl])
- (:use [org.apache.storm testing config clojure util converter])
+ (:use [org.apache.storm.internal clojure])
++ (:use [org.apache.storm testing config util])
(:use [org.apache.storm.daemon common])
- (:require [org.apache.storm [thrift :as thrift]]))
+ (:import [org.apache.storm Thrift])
+ (:import [org.apache.storm.utils Utils]))
(deftest test-basic-topology
(doseq [zmq-on? [true false]]
http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/test/clj/org/apache/storm/cluster_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/cluster_test.clj
index 22c1f80,18e3a80..13198fa
--- a/storm-core/test/clj/org/apache/storm/cluster_test.clj
+++ b/storm-core/test/clj/org/apache/storm/cluster_test.clj
@@@ -30,7 -30,8 +30,8 @@@
(:require [conjure.core])
(:use [conjure core])
(:use [clojure test])
- (:use [org.apache.storm config util testing thrift log converter]))
- (:use [org.apache.storm cluster config util testing log])
++ (:use [org.apache.storm config util testing log converter])
+ (:use [org.apache.storm.internal thrift]))
(defn mk-config [zk-port]
(merge (clojurify-structure (ConfigUtils/readStormConfig))
http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/test/clj/org/apache/storm/messaging/netty_integration_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/messaging/netty_integration_test.clj
index f75a8e3,7fffd34..6a3d3ca
--- a/storm-core/test/clj/org/apache/storm/messaging/netty_integration_test.clj
+++ b/storm-core/test/clj/org/apache/storm/messaging/netty_integration_test.clj
@@@ -1,3 -1,3 +1,4 @@@
++
;; 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
http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/test/clj/org/apache/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/nimbus_test.clj
index 09c4371,ce58f42..3670fd1
--- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj
+++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
@@@ -20,13 -20,12 +20,15 @@@
(:require [org.apache.storm [converter :as converter]])
(:import [org.apache.storm.testing TestWordCounter TestWordSpout TestGlobalCount
TestAggregatesCounter TestPlannerSpout TestPlannerBolt]
- [org.apache.storm.nimbus InMemoryTopologyActionNotifier])
+ [org.apache.storm.nimbus InMemoryTopologyActionNotifier]
+ [org.apache.storm.generated GlobalStreamId]
+ [org.apache.storm Thrift])
(:import [org.apache.storm.testing.staticmocking MockedZookeeper])
(:import [org.apache.storm.scheduler INimbus])
+ (:import [org.mockito Mockito])
+ (:import [org.mockito.exceptions.base MockitoAssertionError])
(:import [org.apache.storm.nimbus ILeaderElector NimbusInfo])
+ (:import [org.apache.storm.testing.staticmocking MockedCluster])
(:import [org.apache.storm.generated Credentials NotAliveException SubmitOptions
TopologyInitialStatus TopologyStatus AlreadyAliveException KillOptions RebalanceOptions
InvalidTopologyException AuthorizationException
@@@ -38,12 -37,10 +40,11 @@@
(:import [org.apache.storm.zookeeper Zookeeper])
(:import [org.apache.commons.io FileUtils]
[org.json.simple JSONValue])
- (:use [org.apache.storm testing MockAutoCred util config log zookeeper])
+ (:import [org.apache.storm.cluster StormClusterStateImpl ClusterStateContext ClusterUtils])
- (:use [org.apache.storm testing MockAutoCred util config log timer converter])
++ (:use [org.apache.storm testing MockAutoCred util config log converter])
(:use [org.apache.storm.daemon common])
(:require [conjure.core])
- (:require [org.apache.storm
- [thrift :as thrift]])
- (:require [org.apache.storm [cluster :as cluster]])
++
(:use [conjure core]))
(defn- from-json
@@@ -1081,10 -1145,11 +1149,11 @@@
STORM-CLUSTER-MODE "local"
STORM-ZOOKEEPER-PORT zk-port
STORM-LOCAL-DIR nimbus-dir}))
- (bind cluster-state (cluster/mk-storm-cluster-state conf))
+ (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
(bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
- (bind topology (thrift/mk-topology
- {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
+ (bind topology (Thrift/buildTopology
+ {"1" (Thrift/prepareSpoutDetails
+ (TestPlannerSpout. true) (Integer. 3))}
{}))
(submit-local-topology nimbus "t1" {} topology)
(submit-local-topology nimbus "t2" {} topology)
@@@ -1153,10 -1218,11 +1222,11 @@@
STORM-CLUSTER-MODE "local"
STORM-ZOOKEEPER-PORT zk-port
STORM-LOCAL-DIR nimbus-dir}))
- (bind cluster-state (cluster/mk-storm-cluster-state conf))
+ (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
(bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
- (bind topology (thrift/mk-topology
- {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
+ (bind topology (Thrift/buildTopology
+ {"1" (Thrift/prepareSpoutDetails
+ (TestPlannerSpout. true) (Integer. 3))}
{}))
(with-open [_ (MockedZookeeper. (proxy [Zookeeper] []
@@@ -1412,17 -1477,17 +1482,16 @@@
(with-open [_ (ConfigUtilsInstaller. fake-cu)
_ (UtilsInstaller. fake-utils)
zk-le (MockedZookeeper. (proxy [Zookeeper] []
- (zkLeaderElectorImpl [conf] nil)))]
+ (zkLeaderElectorImpl [conf] nil)))
+ mocked-cluster (MockedCluster. cluster-utils)]
(stubbing [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
- nimbus/mk-scheduler nil]
- (nimbus/nimbus-data auth-conf fake-inimbus)
- (verify-call-times-for cluster/mk-storm-cluster-state 1)
- (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
- expected-acls))))))
+ nimbus/file-cache-map nil
+ nimbus/mk-blob-cache-map nil
+ nimbus/mk-bloblist-cache-map nil
- mk-timer nil
+ nimbus/mk-scheduler nil]
+ (nimbus/nimbus-data auth-conf fake-inimbus)
+ (.mkStormClusterStateImpl (Mockito/verify cluster-utils (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
+ )))))
(deftest test-file-bogus-download
(with-local-cluster [cluster :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0 TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
@@@ -1453,11 -1518,12 +1522,12 @@@
STORM-CLUSTER-MODE "local"
STORM-ZOOKEEPER-PORT zk-port
STORM-LOCAL-DIR nimbus-dir}))
- (bind cluster-state (cluster/mk-storm-cluster-state conf))
+ (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
(bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
(Time/sleepSecs 1)
- (bind topology (thrift/mk-topology
- {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
+ (bind topology (Thrift/buildTopology
+ {"1" (Thrift/prepareSpoutDetails
+ (TestPlannerSpout. true) (Integer. 3))}
{}))
(submit-local-topology nimbus "t1" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 30} topology)
; make transition for topology t1 to be killed -> nimbus applies this event to cluster state
http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/test/clj/org/apache/storm/supervisor_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/supervisor_test.clj
index 3ebdbcd,ef40c4a..cdd66e4
--- a/storm-core/test/clj/org/apache/storm/supervisor_test.clj
+++ b/storm-core/test/clj/org/apache/storm/supervisor_test.clj
@@@ -22,22 -22,21 +22,21 @@@
(:import [org.apache.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter TestPlannerSpout])
(:import [org.apache.storm.scheduler ISupervisor])
(:import [org.apache.storm.utils Time Utils$UptimeComputer ConfigUtils])
- (:import [org.apache.storm.generated RebalanceOptions])
+ (:import [org.apache.storm.generated RebalanceOptions WorkerResources])
- (:import [org.mockito Matchers Mockito])
+ (:import [org.apache.storm.testing.staticmocking MockedCluster])
(:import [java.util UUID])
++ (:import [org.apache.storm Thrift])
+ (:import [org.mockito Mockito Matchers])
+ (:import [org.mockito.exceptions.base MockitoAssertionError])
(:import [java.io File])
(:import [java.nio.file Files])
- (:import [org.apache.storm.utils Utils IPredicate]
- [org.apache.storm.utils.staticmocking ConfigUtilsInstaller
- UtilsInstaller])
+ (:import [org.apache.storm.utils Utils IPredicate])
+ (:import [org.apache.storm.cluster StormClusterStateImpl ClusterStateContext ClusterUtils]
- [org.apache.storm.utils.staticmocking ConfigUtilsInstaller
- UtilsInstaller])
++ [org.apache.storm.utils.staticmocking ConfigUtilsInstaller UtilsInstaller])
(:import [java.nio.file.attribute FileAttribute])
- (:use [org.apache.storm config testing util timer log converter])
- (:import [org.apache.storm Thrift])
- (:import [org.apache.storm.utils Utils])
- (:use [org.apache.storm config testing util log])
++ (:use [org.apache.storm config testing util log converter])
(:use [org.apache.storm.daemon common])
-- (:require [org.apache.storm.daemon [worker :as worker] [supervisor :as supervisor]]
- [org.apache.storm [thrift :as thrift]])
- [org.apache.storm [cluster :as cluster]])
++ (:require [org.apache.storm.daemon [worker :as worker] [supervisor :as supervisor]])
(:use [conjure core])
(:require [clojure.java.io :as io]))
@@@ -770,66 -772,68 +771,68 @@@
childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
(is (= expected-childopts childopts-with-ids)))))
-(deftest test-retry-read-assignments
- (with-simulated-time-local-cluster [cluster
- :supervisors 0
- :ports-per-supervisor 2
- :daemon-conf {ConfigUtils/NIMBUS_DO_NOT_REASSIGN true
- NIMBUS-MONITOR-FREQ-SECS 10
- TOPOLOGY-MESSAGE-TIMEOUT-SECS 30
- TOPOLOGY-ACKER-EXECUTORS 0}]
- (letlocals
- (bind sup1 (add-supervisor cluster :id "sup1" :ports [1 2 3 4]))
- (bind topology1 (Thrift/buildTopology
- {"1" (Thrift/prepareSpoutDetails
- (TestPlannerSpout. true) (Integer. 2))}
- {}))
- (bind topology2 (Thrift/buildTopology
- {"1" (Thrift/prepareSpoutDetails
- (TestPlannerSpout. true) (Integer. 2))}
- {}))
- (bind state (:storm-cluster-state cluster))
- (bind changed (capture-changed-workers
- (submit-mocked-assignment
- (:nimbus cluster)
- (:storm-cluster-state cluster)
- "topology1"
- {TOPOLOGY-WORKERS 2}
- topology1
- {1 "1"
- 2 "1"}
- {[1 1] ["sup1" 1]
- [2 2] ["sup1" 2]}
- {["sup1" 1] [0.0 0.0 0.0]
- ["sup1" 2] [0.0 0.0 0.0]
- })
- (submit-mocked-assignment
- (:nimbus cluster)
- (:storm-cluster-state cluster)
- "topology2"
- {TOPOLOGY-WORKERS 2}
- topology2
- {1 "1"
- 2 "1"}
- {[1 1] ["sup1" 1]
- [2 2] ["sup1" 2]}
- {["sup1" 1] [0.0 0.0 0.0]
- ["sup1" 2] [0.0 0.0 0.0]
- })
- ;; Instead of sleeping until topology is scheduled, rebalance topology so mk-assignments is called.
- (.rebalance (:nimbus cluster) "topology1" (doto (RebalanceOptions.) (.set_wait_secs 0)))
- ))
- (is (empty? (:launched changed)))
- (bind options (RebalanceOptions.))
- (.set_wait_secs options 0)
- (bind changed (capture-changed-workers
- (.rebalance (:nimbus cluster) "topology2" options)
- (advance-cluster-time cluster 10)
- (heartbeat-workers cluster "sup1" [1 2 3 4])
- (advance-cluster-time cluster 10)
- ))
- (validate-launched-once (:launched changed)
- {"sup1" [1 2]}
- (get-storm-id (:storm-cluster-state cluster) "topology1"))
- (validate-launched-once (:launched changed)
- {"sup1" [3 4]}
- (get-storm-id (:storm-cluster-state cluster) "topology2"))
- )))
+ (deftest test-retry-read-assignments
+ (with-simulated-time-local-cluster [cluster
+ :supervisors 0
+ :ports-per-supervisor 2
+ :daemon-conf {ConfigUtils/NIMBUS_DO_NOT_REASSIGN true
+ NIMBUS-MONITOR-FREQ-SECS 10
+ TOPOLOGY-MESSAGE-TIMEOUT-SECS 30
+ TOPOLOGY-ACKER-EXECUTORS 0}]
+ (letlocals
+ (bind sup1 (add-supervisor cluster :id "sup1" :ports [1 2 3 4]))
- (bind topology1 (thrift/mk-topology
- {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)}
++ (bind topology1 (Thrift/buildTopology
++ {"1" (Thrift/prepareSpoutDetails
++ (TestPlannerSpout. true) (Integer. 2))}
+ {}))
- (bind topology2 (thrift/mk-topology
- {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)}
++ (bind topology2 (Thrift/buildTopology
++ {"1" (Thrift/prepareSpoutDetails
++ (TestPlannerSpout. true) (Integer. 2))}
+ {}))
+ (bind state (:storm-cluster-state cluster))
+ (bind changed (capture-changed-workers
+ (submit-mocked-assignment
+ (:nimbus cluster)
+ (:storm-cluster-state cluster)
+ "topology1"
+ {TOPOLOGY-WORKERS 2}
+ topology1
+ {1 "1"
+ 2 "1"}
+ {[1 1] ["sup1" 1]
+ [2 2] ["sup1" 2]}
+ {["sup1" 1] [0.0 0.0 0.0]
+ ["sup1" 2] [0.0 0.0 0.0]
+ })
+ (submit-mocked-assignment
+ (:nimbus cluster)
+ (:storm-cluster-state cluster)
+ "topology2"
+ {TOPOLOGY-WORKERS 2}
+ topology2
+ {1 "1"
+ 2 "1"}
+ {[1 1] ["sup1" 1]
+ [2 2] ["sup1" 2]}
+ {["sup1" 1] [0.0 0.0 0.0]
+ ["sup1" 2] [0.0 0.0 0.0]
+ })
+ ;; Instead of sleeping until topology is scheduled, rebalance topology so mk-assignments is called.
+ (.rebalance (:nimbus cluster) "topology1" (doto (RebalanceOptions.) (.set_wait_secs 0)))
+ ))
+ (is (empty? (:launched changed)))
+ (bind options (RebalanceOptions.))
+ (.set_wait_secs options 0)
+ (bind changed (capture-changed-workers
+ (.rebalance (:nimbus cluster) "topology2" options)
+ (advance-cluster-time cluster 10)
+ (heartbeat-workers cluster "sup1" [1 2 3 4])
+ (advance-cluster-time cluster 10)
+ ))
+ (validate-launched-once (:launched changed)
+ {"sup1" [1 2]}
+ (get-storm-id (:storm-cluster-state cluster) "topology1"))
+ (validate-launched-once (:launched changed)
+ {"sup1" [3 4]}
+ (get-storm-id (:storm-cluster-state cluster) "topology2"))
+ )))
[09/27] storm git commit: update class hierarchy about cluster
Posted by bo...@apache.org.
update class hierarchy about cluster
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/55b86ca4
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/55b86ca4
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/55b86ca4
Branch: refs/heads/master
Commit: 55b86ca4f0ea02b25701f25f454e537cbf6239d4
Parents: 2a17c45
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Fri Feb 5 13:47:12 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Fri Feb 5 14:17:58 2016 +0800
----------------------------------------------------------------------
conf/defaults.yaml | 2 +-
.../clj/org/apache/storm/command/heartbeats.clj | 4 +-
.../src/clj/org/apache/storm/daemon/common.clj | 5 +-
.../clj/org/apache/storm/daemon/executor.clj | 6 +-
.../src/clj/org/apache/storm/daemon/nimbus.clj | 30 +-
.../clj/org/apache/storm/daemon/supervisor.clj | 12 +-
.../src/clj/org/apache/storm/daemon/worker.clj | 20 +-
.../storm/pacemaker/pacemaker_state_factory.clj | 11 +-
storm-core/src/clj/org/apache/storm/stats.clj | 2 +-
storm-core/src/clj/org/apache/storm/testing.clj | 6 +-
.../jvm/org/apache/storm/callback/Callback.java | 26 -
.../storm/callback/ZKStateChangedCallback.java | 25 +
.../jvm/org/apache/storm/cluster/Cluster.java | 236 -------
.../org/apache/storm/cluster/ClusterState.java | 220 ------
.../storm/cluster/ClusterStateContext.java | 2 +-
.../storm/cluster/ClusterStateFactory.java | 28 -
.../org/apache/storm/cluster/ClusterUtils.java | 249 +++++++
.../storm/cluster/DistributedClusterState.java | 268 --------
.../org/apache/storm/cluster/StateStorage.java | 220 ++++++
.../storm/cluster/StateStorageFactory.java | 28 +
.../apache/storm/cluster/StormClusterState.java | 2 +-
.../storm/cluster/StormClusterStateImpl.java | 664 ++++++++++++++++++
.../storm/cluster/StormZkClusterState.java | 683 -------------------
.../apache/storm/cluster/ZKStateStorage.java | 271 ++++++++
.../storm/cluster/ZKStateStorageFactory.java | 36 +
.../testing/staticmocking/MockedCluster.java | 8 +-
.../org/apache/storm/zookeeper/Zookeeper.java | 22 +-
.../org/apache/storm/integration_test.clj | 4 +-
.../test/clj/org/apache/storm/cluster_test.clj | 124 ++--
.../test/clj/org/apache/storm/nimbus_test.clj | 26 +-
.../clj/org/apache/storm/supervisor_test.clj | 8 +-
.../test/jvm/org/apache/storm/ClusterTest.java | 22 -
32 files changed, 1648 insertions(+), 1622 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 74605bb..b517b90 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -51,7 +51,7 @@ storm.auth.simple-white-list.users: []
storm.auth.simple-acl.users: []
storm.auth.simple-acl.users.commands: []
storm.auth.simple-acl.admins: []
-storm.cluster.state.store: "org.apache.storm.cluster.StormZkClusterState"
+storm.cluster.state.store: "org.apache.storm.cluster.ZKStateStorageFactory"
storm.meta.serialization.delegate: "org.apache.storm.serialization.GzipThriftSerializationDelegate"
storm.codedistributor.class: "org.apache.storm.codedistributor.LocalFileSystemCodeDistributor"
storm.workers.artifacts.dir: "workers-artifacts"
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/command/heartbeats.clj b/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
index 954042f..af86b69 100644
--- a/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
+++ b/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
@@ -22,12 +22,12 @@
[clojure.string :as string])
(:import [org.apache.storm.generated ClusterWorkerHeartbeat]
[org.apache.storm.utils Utils ConfigUtils]
- [org.apache.storm.cluster DistributedClusterState ClusterStateContext])
+ [org.apache.storm.cluster ZKStateStorage ClusterStateContext ClusterUtils])
(:gen-class))
(defn -main [command path & args]
(let [conf (clojurify-structure (ConfigUtils/readStormConfig))
- cluster (DistributedClusterState. conf conf nil (ClusterStateContext.))]
+ cluster (ClusterUtils/mkDistributedClusterState conf conf nil (ClusterStateContext.))]
(println "Command: [" command "]")
(condp = command
"list"
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/clj/org/apache/storm/daemon/common.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/common.clj b/storm-core/src/clj/org/apache/storm/daemon/common.clj
index c9534f4..b144f40 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/common.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/common.clj
@@ -13,7 +13,6 @@
;; 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.
-;TopologyActionOptions TopologyStatus StormBase RebalanceOptions KillOptions
(ns org.apache.storm.daemon.common
(:use [org.apache.storm log config util])
(:import [org.apache.storm.generated StormTopology NodeInfo
@@ -22,7 +21,7 @@
(:import [org.apache.storm.utils Utils ConfigUtils])
(:import [org.apache.storm.task WorkerTopologyContext])
(:import [org.apache.storm Constants])
- (:import [org.apache.storm.cluster StormZkClusterState])
+ (:import [org.apache.storm.cluster StormClusterStateImpl])
(:import [org.apache.storm.metric SystemBolt])
(:import [org.apache.storm.metric EventLoggerBolt])
(:import [org.apache.storm.security.auth IAuthorizer])
@@ -84,7 +83,7 @@
(defn topology-bases [storm-cluster-state]
(let [active-topologies (.activeStorms storm-cluster-state)]
- (into {}
+ (into {}
(dofor [id active-topologies]
[id (.stormBase storm-cluster-state id nil)]
))
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/executor.clj b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
index 7c34c8f..49ae6cf 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@ -34,7 +34,7 @@
(:import [org.apache.storm.daemon Shutdownable])
(:import [org.apache.storm.metric.api IMetric IMetricsConsumer$TaskInfo IMetricsConsumer$DataPoint StateMetric])
(:import [org.apache.storm Config Constants])
- (:import [org.apache.storm.cluster ClusterStateContext DaemonType StormZkClusterState Cluster])
+ (:import [org.apache.storm.cluster ClusterStateContext DaemonType StormClusterStateImpl ClusterUtils])
(:import [org.apache.storm.grouping LoadAwareCustomStreamGrouping LoadAwareShuffleGrouping LoadMapping ShuffleGrouping])
(:import [java.util.concurrent ConcurrentLinkedQueue])
(:require [org.apache.storm [thrift :as thrift] [disruptor :as disruptor] [stats :as stats]])
@@ -208,7 +208,7 @@
(when (<= @interval-errors max-per-interval)
(.reportError (:storm-cluster-state executor) (:storm-id executor) (:component-id executor)
(hostname storm-conf)
- (.getThisWorkerPort (:worker-context executor)) error)
+ (long (.getThisWorkerPort (:worker-context executor))) error)
))))
;; in its own function so that it can be mocked out by tracked topologies
@@ -251,7 +251,7 @@
:batch-transfer-queue batch-transfer->worker
:transfer-fn (mk-executor-transfer-fn batch-transfer->worker storm-conf)
:suicide-fn (:suicide-fn worker)
- :storm-cluster-state (StormZkClusterState. (:cluster-state worker) (Utils/getWorkerACL storm-conf)
+ :storm-cluster-state (ClusterUtils/mkStormClusterState (:state-store worker) (Utils/getWorkerACL storm-conf)
(ClusterStateContext. DaemonType/WORKER))
:type executor-type
;; TODO: should refactor this to be part of the executor specific map (spout or bolt with :common field)
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index 9b00df3..daf5e45 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@ -48,7 +48,7 @@
ProfileRequest ProfileAction NodeInfo])
(:import [org.apache.storm.daemon Shutdownable])
(:import [org.apache.storm.validation ConfigValidation])
- (:import [org.apache.storm.cluster ClusterStateContext DaemonType StormZkClusterState])
+ (:import [org.apache.storm.cluster ClusterStateContext DaemonType StormClusterStateImpl ClusterUtils])
(:use [org.apache.storm util config log timer local-state converter])
(:require [org.apache.storm [converter :as converter]
[stats :as stats]])
@@ -173,7 +173,7 @@
:authorization-handler (mk-authorization-handler (conf NIMBUS-AUTHORIZER) conf)
:impersonation-authorization-handler (mk-authorization-handler (conf NIMBUS-IMPERSONATION-AUTHORIZER) conf)
:submitted-count (atom 0)
- :storm-cluster-state (StormZkClusterState. conf (when
+ :storm-cluster-state (ClusterUtils/mkStormClusterState conf (when
(Utils/isZkAuthenticationConfiguredStormServer
conf)
NIMBUS-ZK-ACLS)
@@ -586,11 +586,11 @@
(defn update-heartbeats! [nimbus storm-id all-executors existing-assignment]
(log-debug "Updating heartbeats for " storm-id " " (pr-str all-executors))
(let [storm-cluster-state (:storm-cluster-state nimbus)
- executor-beats (let [executor-stats-java-map (.executorBeats storm-cluster-state storm-id (.get_executor_node_port (thriftify-assignment existing-assignment)))]
- (->> (clojurify-structure executor-stats-java-map)
- (map (fn [^ExecutorInfo executor-info ^ClusterWorkerHeartbeat cluster-worker-heartbeat]
- {[(.get_task_start executor-info) (.get_task_end executor-info)] (clojurify-zk-worker-hb cluster-worker-heartbeat)}))
- (into {})))
+ executor-beats (let [executor-stats-java-map (.executorBeats storm-cluster-state storm-id (.get_executor_node_port (thriftify-assignment existing-assignment)))
+ executor-stats-clojurify (clojurify-structure executor-stats-java-map)]
+ (->> (dofor [[^ExecutorInfo executor-info ^ClusterWorkerHeartbeat cluster-worker-heartbeat] executor-stats-clojurify]
+ {[(.get_task_start executor-info) (.get_task_end executor-info)] (clojurify-zk-worker-hb cluster-worker-heartbeat)})
+ (apply merge)))
cache (update-heartbeat-cache (@(:heartbeats-cache nimbus) storm-id)
executor-beats
@@ -1332,6 +1332,14 @@
(InvalidTopologyException.
(str "Failed to submit topology. Topology requests more than " workers-allowed " workers."))))))
+(defn nimbus-topology-bases [storm-cluster-state]
+ (let [active-topologies (.activeStorms storm-cluster-state)]
+ (into {}
+ (dofor [id active-topologies]
+ [id (clojurify-storm-base (.stormBase storm-cluster-state id nil))]
+ ))
+ ))
+
(defn- set-logger-timeouts [log-config]
(let [timeout-secs (.get_reset_log_level_timeout_secs log-config)
timeout (time/plus (time/now) (time/secs timeout-secs))]
@@ -1617,7 +1625,7 @@
(log-message "Nimbus setting debug to " enable? " for storm-name '" storm-name "' storm-id '" storm-id "' sampling pct '" spct "'"
(if (not (clojure.string/blank? component-id)) (str " component-id '" component-id "'")))
(locking (:submit-lock nimbus)
- (.updateStorm storm-cluster-state (thriftify-storm-base storm-id storm-base-updates)))))
+ (.updateStorm storm-cluster-state storm-id (thriftify-storm-base storm-base-updates)))))
(^void setWorkerProfiler
[this ^String id ^ProfileRequest profileRequest]
@@ -1804,8 +1812,7 @@
(when-let [version (:version info)] (.set_version sup-sum version))
sup-sum))
nimbus-uptime ((:uptime nimbus))
- javabases (topology-bases storm-cluster-state)
- bases (into {} (dofor [[id base] javabases][id (clojurify-storm-base base)]))
+ bases (nimbus-topology-bases storm-cluster-state)
nimbuses (.nimbuses storm-cluster-state)
;;update the isLeader field for each nimbus summary
@@ -2162,8 +2169,7 @@
(^TopologyHistoryInfo getTopologyHistory [this ^String user]
(let [storm-cluster-state (:storm-cluster-state nimbus)
- javabases (topology-bases storm-cluster-state)
- bases (into {} (dofor [[id base] javabases][id (clojurify-storm-base base)]))
+ bases (topology-bases storm-cluster-state)
assigned-topology-ids (.assignments storm-cluster-state nil)
user-group-match-fn (fn [topo-id user conf]
(let [topology-conf (try-read-storm-conf conf topo-id (:blob-store nimbus))
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
index 079b221..3a83d03 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
@@ -19,7 +19,7 @@
[org.apache.storm.utils LocalState Time Utils ConfigUtils]
[org.apache.storm.daemon Shutdownable]
[org.apache.storm Constants]
- [org.apache.storm.cluster ClusterStateContext DaemonType StormZkClusterState Cluster]
+ [org.apache.storm.cluster ClusterStateContext DaemonType StormClusterStateImpl ClusterUtils]
[java.net JarURLConnection]
[java.net URI]
[org.apache.commons.io FileUtils])
@@ -66,7 +66,9 @@
(if-let [assignment-version (.assignmentVersion storm-cluster-state sid callback)]
(if (= assignment-version recorded-version)
{sid (get assignment-versions sid)}
- {sid (.assignmentInfoWithVersion storm-cluster-state sid callback)})
+ (let [thriftify-assignment-version (.assignmentInfoWithVersion storm-cluster-state sid callback)
+ assignment (clojurify-assignment (:data thriftify-assignment-version))]
+ {sid {:data assignment :version (:version thriftify-assignment-version)}}))
{sid nil})))
(apply merge)
(filter-val not-nil?))
@@ -77,8 +79,7 @@
(if-let [topo-profile-actions (into [] (for [request (.getTopologyProfileRequests storm-cluster-state sid false)] (clojurify-profile-request request)))]
{sid topo-profile-actions}))
(apply merge))]
-
- {:assignments (into {} (for [[k v] new-assignments] [k (clojurify-assignment (:data v))]))
+ {:assignments (into {} (for [[k v] new-assignments] [k (:data v)]))
:profiler-actions new-profiler-actions
:versions new-assignments})))
@@ -317,7 +318,7 @@
:uptime (uptime-computer)
:version STORM-VERSION
:worker-thread-pids-atom (atom {})
- :storm-cluster-state (Cluster/mkStormClusterState conf (when (Utils/isZkAuthenticationConfiguredStormServer conf)
+ :storm-cluster-state (ClusterUtils/mkStormClusterState conf (when (Utils/isZkAuthenticationConfiguredStormServer conf)
SUPERVISOR-ZK-ACLS)
(ClusterStateContext. DaemonType/SUPERVISOR))
:local-state (ConfigUtils/supervisorState conf)
@@ -536,6 +537,7 @@
storm-id->profiler-actions :profiler-actions
versions :versions}
(assignments-snapshot storm-cluster-state sync-callback assignment-versions)
+
storm-code-map (read-storm-code-locations assignments-snapshot)
all-downloaded-storm-ids (set (read-downloaded-storm-ids conf))
existing-assignment (ls-local-assignments local-state)
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/clj/org/apache/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/worker.clj b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
index 85ed37d..a793009 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
@@ -36,7 +36,7 @@
(:import [org.apache.storm.task WorkerTopologyContext])
(:import [org.apache.storm Constants])
(:import [org.apache.storm.security.auth AuthUtils])
- (:import [org.apache.storm.cluster ClusterStateContext DaemonType DistributedClusterState StormZkClusterState])
+ (:import [org.apache.storm.cluster ClusterStateContext DaemonType ZKStateStorage StormClusterStateImpl ClusterUtils])
(:import [javax.security.auth Subject])
(:import [java.security PrivilegedExceptionAction])
(:import [org.apache.logging.log4j LogManager])
@@ -73,7 +73,7 @@
}]
;; do the zookeeper heartbeat
(try
- (.workerHeartbeat (:storm-cluster-state worker) (:storm-id worker) (:assignment-id worker) (:port worker) (thriftify-zk-worker-hb zk-hb))
+ (.workerHeartbeat (:storm-cluster-state worker) (:storm-id worker) (:assignment-id worker) (long (:port worker)) (thriftify-zk-worker-hb zk-hb))
(catch Exception exc
(log-error exc "Worker failed to write heatbeats to ZK or Pacemaker...will retry")))))
@@ -241,7 +241,7 @@
)
:timer-name timer-name))
-(defn worker-data [conf mq-context storm-id assignment-id port worker-id storm-conf cluster-state storm-cluster-state]
+(defn worker-data [conf mq-context storm-id assignment-id port worker-id storm-conf state-store storm-cluster-state]
(let [assignment-versions (atom {})
executors (set (read-worker-executors storm-conf storm-cluster-state storm-id assignment-id port assignment-versions))
transfer-queue (disruptor/disruptor-queue "worker-transfer-queue" (storm-conf TOPOLOGY-TRANSFER-BUFFER-SIZE)
@@ -267,7 +267,7 @@
:assignment-id assignment-id
:port port
:worker-id worker-id
- :cluster-state cluster-state
+ :state-store state-store
:storm-cluster-state storm-cluster-state
;; when worker bootup, worker will start to setup initial connections to
;; other workers. When all connection is ready, we will enable this flag
@@ -596,14 +596,14 @@
(let [storm-conf (ConfigUtils/readSupervisorStormConf conf storm-id)
storm-conf (clojurify-structure (ConfigUtils/overrideLoginConfigWithSystemProperty storm-conf))
acls (Utils/getWorkerACL storm-conf)
- cluster-state (DistributedClusterState. conf storm-conf acls (ClusterStateContext. DaemonType/WORKER))
- storm-cluster-state (StormZkClusterState. cluster-state acls (ClusterStateContext.))
+ state-store (ClusterUtils/mkDistributedClusterState conf storm-conf acls (ClusterStateContext. DaemonType/WORKER))
+ storm-cluster-state (ClusterUtils/mkStormClusterState state-store acls (ClusterStateContext.))
initial-credentials (clojurify-crdentials (.credentials storm-cluster-state storm-id nil))
auto-creds (AuthUtils/GetAutoCredentials storm-conf)
subject (AuthUtils/populateSubject nil auto-creds initial-credentials)]
(Subject/doAs subject (reify PrivilegedExceptionAction
(run [this]
- (let [worker (worker-data conf shared-mq-context storm-id assignment-id port worker-id storm-conf cluster-state storm-cluster-state)
+ (let [worker (worker-data conf shared-mq-context storm-id assignment-id port worker-id storm-conf state-store storm-cluster-state)
heartbeat-fn #(do-heartbeat worker)
;; do this here so that the worker process dies if this fails
@@ -686,10 +686,10 @@
(log-message "Trigger any worker shutdown hooks")
(run-worker-shutdown-hooks worker)
- (.removeWorkerHeartbeat (:storm-cluster-state worker) storm-id assignment-id port)
+ (.removeWorkerHeartbeat (:storm-cluster-state worker) storm-id assignment-id (long port))
(log-message "Disconnecting from storm cluster state context")
(.disconnect (:storm-cluster-state worker))
- (.close (:cluster-state worker))
+ (.close (:state-store worker))
(log-message "Shut down worker " storm-id " " assignment-id " " port))
ret (reify
Shutdownable
@@ -732,7 +732,7 @@
(.topologyLogConfig (:storm-cluster-state worker) storm-id (fn [args] (check-log-config-changed))))
(establish-log-setting-callback)
- (clojurify-crdentials (.credentials (:storm-cluster-state worker) storm-id (fn [args] (check-credentials-changed))))
+ (clojurify-crdentials (.credentials (:storm-cluster-state worker) storm-id (fn [] (check-credentials-changed))))
(schedule-recurring (:refresh-credentials-timer worker) 0 (conf TASK-CREDENTIALS-POLL-SECS)
(fn [& args]
(check-credentials-changed)
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj b/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
index b367b4b..28f792d 100644
--- a/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
+++ b/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
@@ -23,16 +23,17 @@
(:import [org.apache.storm.generated
HBExecutionException HBServerMessageType HBMessage
HBMessageData HBPulse]
- [org.apache.storm.cluster ClusterState DistributedClusterState]
+ [org.apache.storm.cluster ZKStateStorage StateStorage ClusterUtils]
[org.apache.storm.pacemaker PacemakerClient])
- (:gen-class))
+ (:gen-class
+ :implements [org.apache.storm.cluster.StateStorageFactory]))
;; So we can mock the client for testing
(defn makeClient [conf]
(PacemakerClient. conf))
(defn makeZKState [conf auth-conf acls context]
- (DistributedClusterState. conf auth-conf acls context))
+ (ClusterUtils/mkDistributedClusterState conf auth-conf acls context))
(def max-retries 10)
@@ -41,9 +42,9 @@
pacemaker-client (makeClient conf)]
(reify
- ClusterState
+ StateStorage
;; Let these pass through to the zk-state. We only want to handle heartbeats.
- (register [this callback] (.register zk-state callback)) ; need update callback, have questions?? callback is IFn here
+ (register [this callback] (.register zk-state callback))
(unregister [this callback] (.unregister zk-state callback))
(set_ephemeral_node [this path data acls] (.set_ephemeral_node zk-state path data acls))
(create_sequential [this path data acls] (.create_sequential zk-state path data acls))
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/clj/org/apache/storm/stats.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/stats.clj b/storm-core/src/clj/org/apache/storm/stats.clj
index d6bcdc3..0bf1757 100644
--- a/storm-core/src/clj/org/apache/storm/stats.clj
+++ b/storm-core/src/clj/org/apache/storm/stats.clj
@@ -24,7 +24,7 @@
ExecutorAggregateStats SpecificAggregateStats
SpoutAggregateStats TopologyPageInfo TopologyStats])
(:import [org.apache.storm.utils Utils])
- (:import [org.apache.storm.cluster StormZkClusterState])
+ (:import [org.apache.storm.cluster StormClusterStateImpl])
(:import [org.apache.storm.metric.internal MultiCountStatAndMetric MultiLatencyStatAndMetric])
(:use [org.apache.storm log util])
(:use [clojure.math.numeric-tower :only [ceil]]))
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/clj/org/apache/storm/testing.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/testing.clj b/storm-core/src/clj/org/apache/storm/testing.clj
index eb34d36..470a14f 100644
--- a/storm-core/src/clj/org/apache/storm/testing.clj
+++ b/storm-core/src/clj/org/apache/storm/testing.clj
@@ -45,7 +45,7 @@
(:import [org.apache.storm.tuple Tuple])
(:import [org.apache.storm.generated StormTopology])
(:import [org.apache.storm.task TopologyContext])
- (:import [org.apache.storm.cluster DistributedClusterState ClusterStateContext StormZkClusterState])
+ (:import [org.apache.storm.cluster ZKStateStorage ClusterStateContext StormClusterStateImpl ClusterUtils])
(:require [org.apache.storm.messaging.loader :as msg-loader])
(:require [org.apache.storm.daemon.acker :as acker])
(:use [org.apache.storm util thrift config log local-state converter]))
@@ -158,8 +158,8 @@
:port-counter port-counter
:daemon-conf daemon-conf
:supervisors (atom [])
- :state (DistributedClusterState. daemon-conf nil nil (ClusterStateContext.))
- :storm-cluster-state (StormZkClusterState. daemon-conf nil (ClusterStateContext.))
+ :state (ClusterUtils/mkDistributedClusterState daemon-conf nil nil (ClusterStateContext.))
+ :storm-cluster-state (ClusterUtils/mkStormClusterState daemon-conf nil (ClusterStateContext.))
:tmp-dirs (atom [nimbus-tmp zk-tmp])
:zookeeper (if (not-nil? zk-handle) zk-handle)
:shared-context context
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/callback/Callback.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/callback/Callback.java b/storm-core/src/jvm/org/apache/storm/callback/Callback.java
deleted file mode 100644
index a37612d..0000000
--- a/storm-core/src/jvm/org/apache/storm/callback/Callback.java
+++ /dev/null
@@ -1,26 +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.callback;
-
-import clojure.lang.IFn;
-
-// To remove IFn after porting all callbacks to java
-public interface Callback {
- public <T> Object execute(T... args);
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/callback/ZKStateChangedCallback.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/callback/ZKStateChangedCallback.java b/storm-core/src/jvm/org/apache/storm/callback/ZKStateChangedCallback.java
new file mode 100644
index 0000000..75b0e99
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/callback/ZKStateChangedCallback.java
@@ -0,0 +1,25 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.callback;
+
+import org.apache.zookeeper.Watcher;
+
+public interface ZKStateChangedCallback {
+ public void changed(Watcher.Event.EventType type, String path);
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java b/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java
deleted file mode 100644
index 851858f..0000000
--- a/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java
+++ /dev/null
@@ -1,236 +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.cluster;
-
-import org.apache.storm.Config;
-import org.apache.storm.generated.ClusterWorkerHeartbeat;
-import org.apache.storm.generated.ExecutorInfo;
-import org.apache.storm.generated.ExecutorStats;
-import org.apache.storm.generated.ProfileAction;
-import org.apache.storm.utils.Utils;
-import org.apache.zookeeper.ZooDefs;
-import org.apache.zookeeper.data.ACL;
-import org.apache.zookeeper.data.Id;
-import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
-
-
-import java.io.UnsupportedEncodingException;
-import java.net.URLEncoder;
-import java.security.NoSuchAlgorithmException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class Cluster {
-
- public static final String ZK_SEPERATOR = "/";
-
- public static final String ASSIGNMENTS_ROOT = "assignments";
- public static final String CODE_ROOT = "code";
- public static final String STORMS_ROOT = "storms";
- public static final String SUPERVISORS_ROOT = "supervisors";
- public static final String WORKERBEATS_ROOT = "workerbeats";
- public static final String BACKPRESSURE_ROOT = "backpressure";
- public static final String ERRORS_ROOT = "errors";
- public static final String BLOBSTORE_ROOT = "blobstore";
- public static final String BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_ROOT = "blobstoremaxkeysequencenumber";
- public static final String NIMBUSES_ROOT = "nimbuses";
- public static final String CREDENTIALS_ROOT = "credentials";
- public static final String LOGCONFIG_ROOT = "logconfigs";
- public static final String PROFILERCONFIG_ROOT = "profilerconfigs";
-
- public static final String ASSIGNMENTS_SUBTREE;
- public static final String STORMS_SUBTREE;
- public static final String SUPERVISORS_SUBTREE;
- public static final String WORKERBEATS_SUBTREE;
- public static final String BACKPRESSURE_SUBTREE;
- public static final String ERRORS_SUBTREE;
- public static final String BLOBSTORE_SUBTREE;
- public static final String BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_SUBTREE;
- public static final String NIMBUSES_SUBTREE;
- public static final String CREDENTIALS_SUBTREE;
- public static final String LOGCONFIG_SUBTREE;
- public static final String PROFILERCONFIG_SUBTREE;
-
- static {
- ASSIGNMENTS_SUBTREE = ZK_SEPERATOR + ASSIGNMENTS_ROOT;
- STORMS_SUBTREE = ZK_SEPERATOR + STORMS_ROOT;
- SUPERVISORS_SUBTREE = ZK_SEPERATOR + SUPERVISORS_ROOT;
- WORKERBEATS_SUBTREE = ZK_SEPERATOR + WORKERBEATS_ROOT;
- BACKPRESSURE_SUBTREE = ZK_SEPERATOR + BACKPRESSURE_ROOT;
- ERRORS_SUBTREE = ZK_SEPERATOR + ERRORS_ROOT;
- BLOBSTORE_SUBTREE = ZK_SEPERATOR + BLOBSTORE_ROOT;
- BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_SUBTREE = ZK_SEPERATOR + BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_ROOT;
- NIMBUSES_SUBTREE = ZK_SEPERATOR + NIMBUSES_ROOT;
- CREDENTIALS_SUBTREE = ZK_SEPERATOR + CREDENTIALS_ROOT;
- LOGCONFIG_SUBTREE = ZK_SEPERATOR + LOGCONFIG_ROOT;
- PROFILERCONFIG_SUBTREE = ZK_SEPERATOR + PROFILERCONFIG_ROOT;
- }
-
- // A singleton instance allows us to mock delegated static methods in our
- // tests by subclassing.
- private static final Cluster INSTANCE = new Cluster();
- private static Cluster _instance = INSTANCE;
-
- /**
- * Provide an instance of this class for delegates to use. To mock out
- * delegated methods, provide an instance of a subclass that overrides the
- * implementation of the delegated method.
- *
- * @param u a Zookeeper instance
- */
- public static void setInstance(Cluster u) {
- _instance = u;
- }
-
- /**
- * Resets the singleton instance to the default. This is helpful to reset
- * the class to its original functionality when mocking is no longer
- * desired.
- */
- public static void resetInstance() {
- _instance = INSTANCE;
- }
-
- public static List<ACL> mkTopoOnlyAcls(Map topoConf) throws NoSuchAlgorithmException {
- List<ACL> aclList = null;
- String payload = (String)topoConf.get(Config.STORM_ZOOKEEPER_AUTH_PAYLOAD);
- if (Utils.isZkAuthenticationConfiguredStormServer(topoConf)){
- aclList = new ArrayList<>();
- ACL acl1 = ZooDefs.Ids.CREATOR_ALL_ACL.get(0);
- aclList.add(acl1);
- ACL acl2 = new ACL(ZooDefs.Perms.READ, new Id("digest", DigestAuthenticationProvider.generateDigest(payload)));
- aclList.add(acl2);
- }
- return aclList;
- }
-
- public static String supervisorPath(String id) {
- return SUPERVISORS_SUBTREE + ZK_SEPERATOR + id;
- }
-
- public static String assignmentPath(String id) {
- return ASSIGNMENTS_SUBTREE + ZK_SEPERATOR + id;
- }
-
- public static String blobstorePath(String key) {
- return BLOBSTORE_SUBTREE + ZK_SEPERATOR + key;
- }
-
- public static String blobstoreMaxKeySequenceNumberPath(String key) {
- return BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_SUBTREE + ZK_SEPERATOR + key;
- }
-
- public static String nimbusPath(String id) {
- return NIMBUSES_SUBTREE + ZK_SEPERATOR + id;
- }
-
- public static String stormPath(String id) {
- return STORMS_SUBTREE + ZK_SEPERATOR + id;
- }
-
- public static String workerbeatStormRoot(String stormId) {
- return WORKERBEATS_SUBTREE + ZK_SEPERATOR + stormId;
- }
-
- public static String workerbeatPath(String stormId, String node, Long port) {
- return workerbeatStormRoot(stormId) + ZK_SEPERATOR + node + "-" + port;
- }
-
- public static String backpressureStormRoot(String stormId) {
- return BACKPRESSURE_SUBTREE + ZK_SEPERATOR + stormId;
- }
-
- public static String backpressurePath(String stormId, String node, Long port) {
- return backpressureStormRoot(stormId) + ZK_SEPERATOR + node + "-" + port;
- }
-
- public static String errorStormRoot(String stormId) {
- return ERRORS_SUBTREE + ZK_SEPERATOR + stormId;
- }
-
- public static String errorPath(String stormId, String componentId) throws UnsupportedEncodingException {
- return errorStormRoot(stormId) + ZK_SEPERATOR + URLEncoder.encode(componentId, "UTF-8");
- }
-
- public static String lastErrorPath(String stormId, String componentId) throws UnsupportedEncodingException {
- return errorPath(stormId, componentId) + "-last-error";
- }
-
- public static String credentialsPath(String stormId) {
- return CREDENTIALS_SUBTREE + ZK_SEPERATOR + stormId;
- }
-
- public static String logConfigPath(String stormId) {
- return LOGCONFIG_SUBTREE + ZK_SEPERATOR + stormId;
- }
-
- public static String profilerConfigPath(String stormId) {
- return PROFILERCONFIG_SUBTREE + ZK_SEPERATOR + stormId;
- }
-
- public static String profilerConfigPath(String stormId, String host, Long port, ProfileAction requestType) {
- return profilerConfigPath(stormId) + ZK_SEPERATOR + host + "_" + port + "_" + requestType;
- }
-
- public static <T> T maybeDeserialize(byte[] serialized, Class<T> clazz){
- if (serialized != null){
- return Utils.deserialize(serialized, clazz);
- }
- return null;
- }
-
- //Ensures that we only return heartbeats for executors assigned to this worker
- public static Map<ExecutorInfo, ClusterWorkerHeartbeat> convertExecutorBeats(List<ExecutorInfo> executors, ClusterWorkerHeartbeat workerHeartbeat){
- Map<ExecutorInfo, ClusterWorkerHeartbeat> executorWhb = new HashMap<>();
- Map<ExecutorInfo, ExecutorStats> executorStatsMap = workerHeartbeat.get_executor_stats();
- for (ExecutorInfo executor : executors){
- if(executorStatsMap.containsKey(executor)){
- executorWhb.put(executor, workerHeartbeat);
- }
- }
- return executorWhb;
- }
-
- public StormClusterState mkStormClusterStateImpl(Object clusterState, List<ACL> acls, ClusterStateContext context) throws Exception{
- return new StormZkClusterState(clusterState, acls, context);
- }
- public static StormClusterState mkStormClusterState(Object clusterState, List<ACL> acls, ClusterStateContext context) throws Exception{
- return _instance.mkStormClusterStateImpl(clusterState, acls, context);
- }
-
- // TO be remove
- public static <K, V> HashMap<V, List<K>> reverseMap(Map<K, V> map) {
- HashMap<V, List<K>> rtn = new HashMap<V, List<K>>();
- if (map == null) {
- return rtn;
- }
- for (Map.Entry<K, V> entry : map.entrySet()) {
- K key = entry.getKey();
- V val = entry.getValue();
- List<K> list = rtn.get(val);
- if (list == null) {
- list = new ArrayList<K>();
- rtn.put(entry.getValue(), list);
- }
- list.add(key);
- }
- return rtn;
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java
deleted file mode 100644
index e76721b..0000000
--- a/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java
+++ /dev/null
@@ -1,220 +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.cluster;
-
-import clojure.lang.APersistentMap;
-import clojure.lang.IFn;
-import java.util.List;
-
-import org.apache.curator.framework.state.ConnectionStateListener;
-import org.apache.storm.callback.Callback;
-import org.apache.zookeeper.data.ACL;
-
-/**
- * ClusterState provides the API for the pluggable state store used by the
- * Storm daemons. Data is stored in path/value format, and the store supports
- * listing sub-paths at a given path.
- * All data should be available across all nodes with eventual consistency.
- *
- * IMPORTANT NOTE:
- * Heartbeats have different api calls used to interact with them. The root
- * path (/) may or may not be the same as the root path for the other api calls.
- *
- * For example, performing these two calls:
- * set_data("/path", data, acls);
- * void set_worker_hb("/path", heartbeat, acls);
- * may or may not cause a collision in "/path".
- * Never use the same paths with the *_hb* methods as you do with the others.
- */
-public interface ClusterState {
-
- /**
- * Registers a callback function that gets called when CuratorEvents happen.
- * @param callback is a clojure IFn that accepts the type - translated to
- * clojure keyword as in zookeeper - and the path: (callback type path)
- * @return is an id that can be passed to unregister(...) to unregister the
- * callback.
- */
- String register(Callback callback);
-
- /**
- * Unregisters a callback function that was registered with register(...).
- * @param id is the String id that was returned from register(...).
- */
- void unregister(String id);
-
- /**
- * Path will be appended with a monotonically increasing integer, a new node
- * will be created there, and data will be put at that node.
- * @param path The path that the monotonically increasing integer suffix will
- * be added to.
- * @param data The data that will be written at the suffixed path's node.
- * @param acls The acls to apply to the path. May be null.
- * @return The path with the integer suffix appended.
- */
- String create_sequential(String path, byte[] data, List<ACL> acls);
-
- /**
- * Creates nodes for path and all its parents. Path elements are separated by
- * a "/", as in *nix filesystem notation. Equivalent to mkdir -p in *nix.
- * @param path The path to create, along with all its parents.
- * @param acls The acls to apply to the path. May be null.
- * @return path
- */
- void mkdirs(String path, List<ACL> acls);
-
- /**
- * Deletes the node at a given path, and any child nodes that may exist.
- * @param path The path to delete
- */
- void delete_node(String path);
-
- /**
- * Creates an ephemeral node at path. Ephemeral nodes are destroyed
- * by the store when the client disconnects.
- * @param path The path where a node will be created.
- * @param data The data to be written at the node.
- * @param acls The acls to apply to the path. May be null.
- */
- void set_ephemeral_node(String path, byte[] data, List<ACL> acls);
-
- /**
- * Gets the 'version' of the node at a path. Optionally sets a watch
- * on that node. The version should increase whenever a write happens.
- * @param path The path to get the version of.
- * @param watch Whether or not to set a watch on the path. Watched paths
- * emit events which are consumed by functions registered with the
- * register method. Very useful for catching updates to nodes.
- * @return The integer version of this node.
- */
- Integer get_version(String path, boolean watch) throws Exception;
-
- /**
- * Check if a node exists and optionally set a watch on the path.
- * @param path The path to check for the existence of a node.
- * @param watch Whether or not to set a watch on the path. Watched paths
- * emit events which are consumed by functions registered with the
- * register method. Very useful for catching updates to nodes.
- * @return Whether or not a node exists at path.
- */
- boolean node_exists(String path, boolean watch);
-
- /**
- * Get a list of paths of all the child nodes which exist immediately
- * under path.
- * @param path The path to look under
- * @param watch Whether or not to set a watch on the path. Watched paths
- * emit events which are consumed by functions registered with the
- * register method. Very useful for catching updates to nodes.
- * @return list of string paths under path.
- */
- List<String> get_children(String path, boolean watch);
-
- /**
- * Close the connection to the data store.
- */
- void close();
-
- /**
- * Set the value of the node at path to data.
- * @param path The path whose node we want to set.
- * @param data The data to put in the node.
- * @param acls The acls to apply to the path. May be null.
- */
- void set_data(String path, byte[] data, List<ACL> acls);
-
- /**
- * Get the data from the node at path
- * @param path The path to look under
- * @param watch Whether or not to set a watch on the path. Watched paths
- * emit events which are consumed by functions registered with the
- * register method. Very useful for catching updates to nodes.
- * @return The data at the node.
- */
- byte[] get_data(String path, boolean watch);
-
- /**
- * Get the data at the node along with its version. Data is returned
- * in an APersistentMap with clojure keyword keys :data and :version.
- * @param path The path to look under
- * @param watch Whether or not to set a watch on the path. Watched paths
- * emit events which are consumed by functions registered with the
- * register method. Very useful for catching updates to nodes.
- * @return An APersistentMap in the form {:data data :version version}
- */
- APersistentMap get_data_with_version(String path, boolean watch);
-
- /**
- * Write a worker heartbeat at the path.
- * @param path The path whose node we want to set.
- * @param data The data to put in the node.
- * @param acls The acls to apply to the path. May be null.
- */
- void set_worker_hb(String path, byte[] data, List<ACL> acls);
-
- /**
- * Get the heartbeat from the node at path
- * @param path The path to look under
- * @param watch Whether or not to set a watch on the path. Watched paths
- * emit events which are consumed by functions registered with the
- * register method. Very useful for catching updates to nodes.
- * @return The heartbeat at the node.
- */
- byte[] get_worker_hb(String path, boolean watch);
-
- /**
- * Get a list of paths of all the child nodes which exist immediately
- * under path. This is similar to get_children, but must be used for
- * any nodes
- * @param path The path to look under
- * @param watch Whether or not to set a watch on the path. Watched paths
- * emit events which are consumed by functions registered with the
- * register method. Very useful for catching updates to nodes.
- * @return list of string paths under path.
- */
- List<String> get_worker_hb_children(String path, boolean watch);
-
- /**
- * Deletes the heartbeat at a given path, and any child nodes that may exist.
- * @param path The path to delete.
- */
- void delete_worker_hb(String path);
-
- /**
- * Add a ClusterStateListener to the connection.
- * @param listener A ClusterStateListener to handle changing cluster state
- * events.
- */
- void add_listener(final ConnectionStateListener listener);
-
- /**
- * Force consistency on a path. Any writes committed on the path before
- * this call will be completely propagated when it returns.
- * @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/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/ClusterStateContext.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ClusterStateContext.java b/storm-core/src/jvm/org/apache/storm/cluster/ClusterStateContext.java
index 997bdc3..9ad6a92 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ClusterStateContext.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ClusterStateContext.java
@@ -19,7 +19,7 @@
package org.apache.storm.cluster;
/**
- * This class is intended to provide runtime-context to ClusterStateFactory
+ * This class is intended to provide runtime-context to StateStorageFactory
* implementors, giving information such as what daemon is creating it.
*/
public class ClusterStateContext {
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/ClusterStateFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ClusterStateFactory.java b/storm-core/src/jvm/org/apache/storm/cluster/ClusterStateFactory.java
deleted file mode 100644
index 6474d82..0000000
--- a/storm-core/src/jvm/org/apache/storm/cluster/ClusterStateFactory.java
+++ /dev/null
@@ -1,28 +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.cluster;
-
-import clojure.lang.APersistentMap;
-import java.util.List;
-import org.apache.zookeeper.data.ACL;
-
-public interface ClusterStateFactory {
-
- ClusterState mkState(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context);
-
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java b/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
new file mode 100644
index 0000000..9fd36ca
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
@@ -0,0 +1,249 @@
+/**
+ * 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 org.apache.storm.Config;
+import org.apache.storm.generated.ClusterWorkerHeartbeat;
+import org.apache.storm.generated.ExecutorInfo;
+import org.apache.storm.generated.ExecutorStats;
+import org.apache.storm.generated.ProfileAction;
+import org.apache.storm.utils.Utils;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
+import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLEncoder;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class ClusterUtils {
+
+ public static final String ZK_SEPERATOR = "/";
+
+ public static final String ASSIGNMENTS_ROOT = "assignments";
+ public static final String CODE_ROOT = "code";
+ public static final String STORMS_ROOT = "storms";
+ public static final String SUPERVISORS_ROOT = "supervisors";
+ public static final String WORKERBEATS_ROOT = "workerbeats";
+ public static final String BACKPRESSURE_ROOT = "backpressure";
+ public static final String ERRORS_ROOT = "errors";
+ public static final String BLOBSTORE_ROOT = "blobstore";
+ public static final String BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_ROOT = "blobstoremaxkeysequencenumber";
+ public static final String NIMBUSES_ROOT = "nimbuses";
+ public static final String CREDENTIALS_ROOT = "credentials";
+ public static final String LOGCONFIG_ROOT = "logconfigs";
+ public static final String PROFILERCONFIG_ROOT = "profilerconfigs";
+
+ public static final String ASSIGNMENTS_SUBTREE = ZK_SEPERATOR + ASSIGNMENTS_ROOT;
+ public static final String STORMS_SUBTREE = ZK_SEPERATOR + STORMS_ROOT;
+ public static final String SUPERVISORS_SUBTREE = ZK_SEPERATOR + SUPERVISORS_ROOT;
+ public static final String WORKERBEATS_SUBTREE = ZK_SEPERATOR + WORKERBEATS_ROOT;
+ public static final String BACKPRESSURE_SUBTREE = ZK_SEPERATOR + BACKPRESSURE_ROOT;
+ public static final String ERRORS_SUBTREE = ZK_SEPERATOR + ERRORS_ROOT;
+ public static final String BLOBSTORE_SUBTREE = ZK_SEPERATOR + BLOBSTORE_ROOT;
+ public static final String BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_SUBTREE = ZK_SEPERATOR + BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_ROOT;
+ public static final String NIMBUSES_SUBTREE = ZK_SEPERATOR + NIMBUSES_ROOT;
+ public static final String CREDENTIALS_SUBTREE = ZK_SEPERATOR + CREDENTIALS_ROOT;
+ public static final String LOGCONFIG_SUBTREE = ZK_SEPERATOR + LOGCONFIG_ROOT;
+ public static final String PROFILERCONFIG_SUBTREE = ZK_SEPERATOR + PROFILERCONFIG_ROOT;
+
+ // A singleton instance allows us to mock delegated static methods in our
+ // tests by subclassing.
+ private static final ClusterUtils INSTANCE = new ClusterUtils();
+ private static ClusterUtils _instance = INSTANCE;
+
+ /**
+ * Provide an instance of this class for delegates to use. To mock out delegated methods, provide an instance of a subclass that overrides the
+ * implementation of the delegated method.
+ *
+ * @param u a Cluster instance
+ */
+ public static void setInstance(ClusterUtils u) {
+ _instance = u;
+ }
+
+ /**
+ * Resets the singleton instance to the default. This is helpful to reset the class to its original functionality when mocking is no longer desired.
+ */
+ public static void resetInstance() {
+ _instance = INSTANCE;
+ }
+
+ public static List<ACL> mkTopoOnlyAcls(Map topoConf) throws NoSuchAlgorithmException {
+ List<ACL> aclList = null;
+ String payload = (String) topoConf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD);
+ if (Utils.isZkAuthenticationConfiguredStormServer(topoConf)) {
+ aclList = new ArrayList<>();
+ ACL acl1 = ZooDefs.Ids.CREATOR_ALL_ACL.get(0);
+ aclList.add(acl1);
+ ACL acl2 = new ACL(ZooDefs.Perms.READ, new Id("digest", DigestAuthenticationProvider.generateDigest(payload)));
+ aclList.add(acl2);
+ }
+ return aclList;
+ }
+
+ public static String supervisorPath(String id) {
+ return SUPERVISORS_SUBTREE + ZK_SEPERATOR + id;
+ }
+
+ public static String assignmentPath(String id) {
+ return ASSIGNMENTS_SUBTREE + ZK_SEPERATOR + id;
+ }
+
+ public static String blobstorePath(String key) {
+ return BLOBSTORE_SUBTREE + ZK_SEPERATOR + key;
+ }
+
+ public static String blobstoreMaxKeySequenceNumberPath(String key) {
+ return BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_SUBTREE + ZK_SEPERATOR + key;
+ }
+
+ public static String nimbusPath(String id) {
+ return NIMBUSES_SUBTREE + ZK_SEPERATOR + id;
+ }
+
+ public static String stormPath(String id) {
+ return STORMS_SUBTREE + ZK_SEPERATOR + id;
+ }
+
+ public static String workerbeatStormRoot(String stormId) {
+ return WORKERBEATS_SUBTREE + ZK_SEPERATOR + stormId;
+ }
+
+ public static String workerbeatPath(String stormId, String node, Long port) {
+ return workerbeatStormRoot(stormId) + ZK_SEPERATOR + node + "-" + port;
+ }
+
+ public static String backpressureStormRoot(String stormId) {
+ return BACKPRESSURE_SUBTREE + ZK_SEPERATOR + stormId;
+ }
+
+ public static String backpressurePath(String stormId, String node, Long port) {
+ return backpressureStormRoot(stormId) + ZK_SEPERATOR + node + "-" + port;
+ }
+
+ public static String errorStormRoot(String stormId) {
+ return ERRORS_SUBTREE + ZK_SEPERATOR + stormId;
+ }
+
+ public static String errorPath(String stormId, String componentId) {
+ try {
+ return errorStormRoot(stormId) + ZK_SEPERATOR + URLEncoder.encode(componentId, "UTF-8");
+ } catch (UnsupportedEncodingException e) {
+ throw Utils.wrapInRuntime(e);
+ }
+ }
+
+ public static String lastErrorPath(String stormId, String componentId) {
+ return errorPath(stormId, componentId) + "-last-error";
+ }
+
+ public static String credentialsPath(String stormId) {
+ return CREDENTIALS_SUBTREE + ZK_SEPERATOR + stormId;
+ }
+
+ public static String logConfigPath(String stormId) {
+ return LOGCONFIG_SUBTREE + ZK_SEPERATOR + stormId;
+ }
+
+ public static String profilerConfigPath(String stormId) {
+ return PROFILERCONFIG_SUBTREE + ZK_SEPERATOR + stormId;
+ }
+
+ public static String profilerConfigPath(String stormId, String host, Long port, ProfileAction requestType) {
+ return profilerConfigPath(stormId) + ZK_SEPERATOR + host + "_" + port + "_" + requestType;
+ }
+
+ public static <T> T maybeDeserialize(byte[] serialized, Class<T> clazz) {
+ if (serialized != null) {
+ return Utils.deserialize(serialized, clazz);
+ }
+ return null;
+ }
+
+ // Ensures that we only return heartbeats for executors assigned to this worker
+ public static Map<ExecutorInfo, ClusterWorkerHeartbeat> convertExecutorBeats(List<ExecutorInfo> executors, ClusterWorkerHeartbeat workerHeartbeat) {
+ Map<ExecutorInfo, ClusterWorkerHeartbeat> executorWhb = new HashMap<>();
+ Map<ExecutorInfo, ExecutorStats> executorStatsMap = workerHeartbeat.get_executor_stats();
+ for (ExecutorInfo executor : executors) {
+ if (executorStatsMap.containsKey(executor)) {
+ executorWhb.put(executor, workerHeartbeat);
+ }
+ }
+ return executorWhb;
+ }
+
+ public StormClusterState mkStormClusterStateImpl(Object StateStorage, List<ACL> acls, ClusterStateContext context) throws Exception {
+ if (StateStorage instanceof StateStorage) {
+ return new StormClusterStateImpl((StateStorage) StateStorage, acls, context, false);
+ } else {
+ StateStorage Storage = _instance.mkDistributedClusterStateImpl((APersistentMap) StateStorage, (APersistentMap) StateStorage, acls, context);
+ return new StormClusterStateImpl(Storage, acls, context, true);
+ }
+
+ }
+
+ public StateStorage mkDistributedClusterStateImpl(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context)
+ throws Exception {
+ String className = null;
+ StateStorage stateStorage = null;
+ if (config.get(Config.STORM_CLUSTER_STATE_STORE) != null) {
+ className = (String) config.get(Config.STORM_CLUSTER_STATE_STORE);
+ } else {
+ className = "org.apache.storm.cluster.ZKStateStorageFactory";
+ }
+ Class clazz = Class.forName(className);
+ StateStorageFactory storageFactory = (StateStorageFactory) clazz.newInstance();
+ stateStorage = storageFactory.mkState(config, auth_conf, acls, context);
+ return stateStorage;
+ }
+
+ public static StateStorage mkDistributedClusterState(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context)
+ throws Exception {
+ return _instance.mkDistributedClusterStateImpl(config, auth_conf, acls, context);
+ }
+
+ public static StormClusterState mkStormClusterState(Object StateStorage, List<ACL> acls, ClusterStateContext context) throws Exception {
+ return _instance.mkStormClusterStateImpl(StateStorage, acls, context);
+ }
+
+ // TO be remove
+ public static <K, V> HashMap<V, List<K>> reverseMap(Map<K, V> map) {
+ HashMap<V, List<K>> rtn = new HashMap<V, List<K>>();
+ if (map == null) {
+ return rtn;
+ }
+ for (Map.Entry<K, V> entry : map.entrySet()) {
+ K key = entry.getKey();
+ V val = entry.getValue();
+ List<K> list = rtn.get(val);
+ if (list == null) {
+ list = new ArrayList<K>();
+ rtn.put(entry.getValue(), list);
+ }
+ list.add(key);
+ }
+ return rtn;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java
deleted file mode 100644
index 1bd534e..0000000
--- a/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java
+++ /dev/null
@@ -1,268 +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.cluster;
-
-import clojure.lang.APersistentMap;
-import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.state.*;
-import org.apache.curator.framework.state.ConnectionState;
-import org.apache.storm.Config;
-import org.apache.storm.callback.Callback;
-import org.apache.storm.callback.WatcherCallBack;
-import org.apache.storm.utils.Utils;
-import org.apache.storm.zookeeper.Zookeeper;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.data.ACL;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-public class DistributedClusterState implements ClusterState {
-
- private static Logger LOG = LoggerFactory.getLogger(DistributedClusterState.class);
-
- private ConcurrentHashMap<String, Callback> callbacks = new ConcurrentHashMap<String, Callback>();
- private CuratorFramework zkWriter;
- private CuratorFramework zkReader;
- private AtomicBoolean active;
-
- private boolean isNimbus;
- private Map authConf;
- private Map<Object, Object> conf;
-
- public DistributedClusterState(Map<Object, Object> conf, Map authConf, List<ACL> acls, ClusterStateContext context) throws Exception {
- this.conf = conf;
- this.authConf = authConf;
- if (context.getDaemonType().equals(DaemonType.NIMBUS)) this.isNimbus = true;
-
- // just mkdir STORM_ZOOKEEPER_ROOT dir
- CuratorFramework zkTemp = mkZk();
- String rootPath = String.valueOf(conf.get(Config.STORM_ZOOKEEPER_ROOT));
- Zookeeper.mkdirs(zkTemp, rootPath, acls);
- zkTemp.close();
-
- active = new AtomicBoolean(true);
- zkWriter = mkZk(new WatcherCallBack() {
- @Override
- public void execute(Watcher.Event.KeeperState state, Watcher.Event.EventType type, String path) {
- if (active.get()) {
- if (!(state.equals(Watcher.Event.KeeperState.SyncConnected))) {
- LOG.warn("Received event {} : {}: {} with disconnected Zookeeper.", state, type, path);
- } else {
- LOG.info("Received event {} : {} : {}", state, type, path);
- }
-
- if (!type.equals(Watcher.Event.EventType.None)) {
- for (Map.Entry<String, Callback> e : callbacks.entrySet()) {
- Callback fn = e.getValue();
- fn.execute(type, path);
- }
- }
- }
- }
- });
- if (isNimbus) {
- zkReader = mkZk(new WatcherCallBack() {
- @Override
- public void execute(Watcher.Event.KeeperState state, Watcher.Event.EventType type, String path) {
- if (active.get()) {
- if (!(state.equals(Watcher.Event.KeeperState.SyncConnected))) {
- LOG.warn("Received event {} : {}: {} with disconnected Zookeeper.", state, type, path);
- } else {
- LOG.info("Received event {} : {} : {}", state, type, path);
- }
-
- if (!type.equals(Watcher.Event.EventType.None)) {
- for (Map.Entry<String, Callback> e : callbacks.entrySet()) {
- Callback fn = e.getValue();
- fn.execute(type, path);
- }
- }
- }
- }
- });
- } else {
- zkReader = zkWriter;
- }
-
- }
-
- @SuppressWarnings("unchecked")
- private CuratorFramework mkZk() throws IOException {
- return Zookeeper.mkClient(conf, (List<String>) conf.get(Config.STORM_ZOOKEEPER_SERVERS), conf.get(Config.STORM_ZOOKEEPER_PORT), "", authConf);
- }
-
- @SuppressWarnings("unchecked")
- private CuratorFramework mkZk(WatcherCallBack watcher) throws NumberFormatException, IOException {
- return Zookeeper.mkClient(conf, (List<String>) conf.get(Config.STORM_ZOOKEEPER_SERVERS), conf.get(Config.STORM_ZOOKEEPER_PORT),
- String.valueOf(conf.get(Config.STORM_ZOOKEEPER_ROOT)), watcher, authConf);
- }
-
- @Override
- public void delete_node_blobstore(String path, String nimbusHostPortInfo) {
-
- }
-
- @Override
- public String register( Callback callback) {
- String id = UUID.randomUUID().toString();
- this.callbacks.put(id,callback);
- return id;
- }
-
- @Override
- public void unregister(String id) {
- this.callbacks.remove(id);
- }
-
- @Override
- public String create_sequential(String path, byte[] data, List<ACL> acls) {
- return Zookeeper.createNode(zkWriter, path, data, CreateMode.EPHEMERAL_SEQUENTIAL, acls);
- }
-
- @Override
- public void mkdirs(String path, List<ACL> acls) {
- Zookeeper.mkdirs(zkWriter, path, acls);
- }
-
- @Override
- public void delete_node(String path) {
- Zookeeper.deleteNode(zkWriter, path);
- }
-
- @Override
- public void set_ephemeral_node(String path, byte[] data, List<ACL> acls) {
- Zookeeper.mkdirs(zkWriter, parentPath(path), acls);
- if (Zookeeper.exists(zkWriter, path, false)) {
- try {
- Zookeeper.setData(zkWriter, path, data);
- } catch (RuntimeException e) {
- if (Utils.exceptionCauseIsInstanceOf(KeeperException.NodeExistsException.class, e)) {
- Zookeeper.createNode(zkWriter, path, data, CreateMode.EPHEMERAL, acls);
- } else {
- throw e;
- }
- }
-
- } else {
- Zookeeper.createNode(zkWriter, path, data, CreateMode.EPHEMERAL, acls);
- }
- }
-
- @Override
- public Integer get_version(String path, boolean watch) throws Exception {
- Integer ret = Zookeeper.getVersion(zkReader, path, watch);
- return ret;
- }
-
- @Override
- public boolean node_exists(String path, boolean watch) {
- return Zookeeper.existsNode(zkWriter, path, watch);
- }
-
- @Override
- public List<String> get_children(String path, boolean watch) {
- return Zookeeper.getChildren(zkReader, path, watch);
- }
-
- @Override
- public void close() {
- this.active.set(false);
- zkWriter.close();
- if (isNimbus) {
- zkReader.close();
- }
- }
-
- @Override
- public void set_data(String path, byte[] data, List<ACL> acls) {
- if (Zookeeper.exists(zkWriter, path, false)) {
- Zookeeper.setData(zkWriter, path, data);
- } else {
- Zookeeper.mkdirs(zkWriter, parentPath(path), acls);
- Zookeeper.createNode(zkWriter, path, data, CreateMode.PERSISTENT, acls);
- }
- }
-
- @Override
- public byte[] get_data(String path, boolean watch) {
- byte[] ret = null;
-
- ret = Zookeeper.getData(zkReader, path, watch);
-
- return ret;
- }
-
- @Override
- public APersistentMap get_data_with_version(String path, boolean watch) {
- return Zookeeper.getDataWithVersion(zkReader, path, watch);
- }
-
- @Override
- public void set_worker_hb(String path, byte[] data, List<ACL> acls) {
- set_data(path, data, acls);
- }
-
- @Override
- public byte[] get_worker_hb(String path, boolean watch) {
- return Zookeeper.getData(zkReader, path, watch);
- }
-
- @Override
- public List<String> get_worker_hb_children(String path, boolean watch) {
- return get_children(path, watch);
- }
-
- @Override
- public void delete_worker_hb(String path) {
- delete_node(path);
- }
-
- @Override
- public void add_listener(final ConnectionStateListener listener) {
- Zookeeper.addListener(zkReader, new ConnectionStateListener() {
- @Override
- public void stateChanged(CuratorFramework curatorFramework, ConnectionState connectionState) {
- listener.stateChanged(curatorFramework, connectionState);
- }
- });
- }
-
- @Override
- public void sync_path(String path) {
- Zookeeper.syncPath(zkWriter, path);
- }
-
- // To be remove when finished port Util.clj
- public static String parentPath(String path) {
- List<String> toks = Zookeeper.tokenizePath(path);
- int size = toks.size();
- if (size > 0) {
- toks.remove(size - 1);
- }
- return Zookeeper.toksToPath(toks);
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/StateStorage.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StateStorage.java b/storm-core/src/jvm/org/apache/storm/cluster/StateStorage.java
new file mode 100644
index 0000000..8895cd1
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StateStorage.java
@@ -0,0 +1,220 @@
+/**
+ * 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 java.util.List;
+
+import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.storm.callback.ZKStateChangedCallback;
+import org.apache.zookeeper.data.ACL;
+
+/**
+ * StateStorage provides the API for the pluggable state store used by the
+ * Storm daemons. Data is stored in path/value format, and the store supports
+ * listing sub-paths at a given path.
+ * All data should be available across all nodes with eventual consistency.
+ *
+ * IMPORTANT NOTE:
+ * Heartbeats have different api calls used to interact with them. The root
+ * path (/) may or may not be the same as the root path for the other api calls.
+ *
+ * For example, performing these two calls:
+ * set_data("/path", data, acls);
+ * void set_worker_hb("/path", heartbeat, acls);
+ * may or may not cause a collision in "/path".
+ * Never use the same paths with the *_hb* methods as you do with the others.
+ */
+public interface StateStorage {
+
+ /**
+ * Registers a callback function that gets called when CuratorEvents happen.
+ * @param callback is a clojure IFn that accepts the type - translated to
+ * clojure keyword as in zookeeper - and the path: (callback type path)
+ * @return is an id that can be passed to unregister(...) to unregister the
+ * callback.
+ */
+ String register(ZKStateChangedCallback callback);
+
+ /**
+ * Unregisters a callback function that was registered with register(...).
+ * @param id is the String id that was returned from register(...).
+ */
+ void unregister(String id);
+
+ /**
+ * Path will be appended with a monotonically increasing integer, a new node
+ * will be created there, and data will be put at that node.
+ * @param path The path that the monotonically increasing integer suffix will
+ * be added to.
+ * @param data The data that will be written at the suffixed path's node.
+ * @param acls The acls to apply to the path. May be null.
+ * @return The path with the integer suffix appended.
+ */
+ String create_sequential(String path, byte[] data, List<ACL> acls);
+
+ /**
+ * Creates nodes for path and all its parents. Path elements are separated by
+ * a "/", as in *nix filesystem notation. Equivalent to mkdir -p in *nix.
+ * @param path The path to create, along with all its parents.
+ * @param acls The acls to apply to the path. May be null.
+ * @return path
+ */
+ void mkdirs(String path, List<ACL> acls);
+
+ /**
+ * Deletes the node at a given path, and any child nodes that may exist.
+ * @param path The path to delete
+ */
+ void delete_node(String path);
+
+ /**
+ * Creates an ephemeral node at path. Ephemeral nodes are destroyed
+ * by the store when the client disconnects.
+ * @param path The path where a node will be created.
+ * @param data The data to be written at the node.
+ * @param acls The acls to apply to the path. May be null.
+ */
+ void set_ephemeral_node(String path, byte[] data, List<ACL> acls);
+
+ /**
+ * Gets the 'version' of the node at a path. Optionally sets a watch
+ * on that node. The version should increase whenever a write happens.
+ * @param path The path to get the version of.
+ * @param watch Whether or not to set a watch on the path. Watched paths
+ * emit events which are consumed by functions registered with the
+ * register method. Very useful for catching updates to nodes.
+ * @return The integer version of this node.
+ */
+ Integer get_version(String path, boolean watch) throws Exception;
+
+ /**
+ * Check if a node exists and optionally set a watch on the path.
+ * @param path The path to check for the existence of a node.
+ * @param watch Whether or not to set a watch on the path. Watched paths
+ * emit events which are consumed by functions registered with the
+ * register method. Very useful for catching updates to nodes.
+ * @return Whether or not a node exists at path.
+ */
+ boolean node_exists(String path, boolean watch);
+
+ /**
+ * Get a list of paths of all the child nodes which exist immediately
+ * under path.
+ * @param path The path to look under
+ * @param watch Whether or not to set a watch on the path. Watched paths
+ * emit events which are consumed by functions registered with the
+ * register method. Very useful for catching updates to nodes.
+ * @return list of string paths under path.
+ */
+ List<String> get_children(String path, boolean watch);
+
+ /**
+ * Close the connection to the data store.
+ */
+ void close();
+
+ /**
+ * Set the value of the node at path to data.
+ * @param path The path whose node we want to set.
+ * @param data The data to put in the node.
+ * @param acls The acls to apply to the path. May be null.
+ */
+ void set_data(String path, byte[] data, List<ACL> acls);
+
+ /**
+ * Get the data from the node at path
+ * @param path The path to look under
+ * @param watch Whether or not to set a watch on the path. Watched paths
+ * emit events which are consumed by functions registered with the
+ * register method. Very useful for catching updates to nodes.
+ * @return The data at the node.
+ */
+ byte[] get_data(String path, boolean watch);
+
+ /**
+ * Get the data at the node along with its version. Data is returned
+ * in an APersistentMap with clojure keyword keys :data and :version.
+ * @param path The path to look under
+ * @param watch Whether or not to set a watch on the path. Watched paths
+ * emit events which are consumed by functions registered with the
+ * register method. Very useful for catching updates to nodes.
+ * @return An APersistentMap in the form {:data data :version version}
+ */
+ APersistentMap get_data_with_version(String path, boolean watch);
+
+ /**
+ * Write a worker heartbeat at the path.
+ * @param path The path whose node we want to set.
+ * @param data The data to put in the node.
+ * @param acls The acls to apply to the path. May be null.
+ */
+ void set_worker_hb(String path, byte[] data, List<ACL> acls);
+
+ /**
+ * Get the heartbeat from the node at path
+ * @param path The path to look under
+ * @param watch Whether or not to set a watch on the path. Watched paths
+ * emit events which are consumed by functions registered with the
+ * register method. Very useful for catching updates to nodes.
+ * @return The heartbeat at the node.
+ */
+ byte[] get_worker_hb(String path, boolean watch);
+
+ /**
+ * Get a list of paths of all the child nodes which exist immediately
+ * under path. This is similar to get_children, but must be used for
+ * any nodes
+ * @param path The path to look under
+ * @param watch Whether or not to set a watch on the path. Watched paths
+ * emit events which are consumed by functions registered with the
+ * register method. Very useful for catching updates to nodes.
+ * @return list of string paths under path.
+ */
+ List<String> get_worker_hb_children(String path, boolean watch);
+
+ /**
+ * Deletes the heartbeat at a given path, and any child nodes that may exist.
+ * @param path The path to delete.
+ */
+ void delete_worker_hb(String path);
+
+ /**
+ * Add a StateStorageListener to the connection.
+ * @param listener A StateStorageListener to handle changing cluster state
+ * events.
+ */
+ void add_listener(final ConnectionStateListener listener);
+
+ /**
+ * Force consistency on a path. Any writes committed on the path before
+ * this call will be completely propagated when it returns.
+ * @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/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java b/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
new file mode 100644
index 0000000..9803dff
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
@@ -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.
+ */
+package org.apache.storm.cluster;
+
+import clojure.lang.APersistentMap;
+import java.util.List;
+import org.apache.zookeeper.data.ACL;
+
+public interface StateStorageFactory {
+
+ StateStorage mkState(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context);
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
index ede2ba3..58b125b 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
@@ -114,7 +114,7 @@ public interface StormClusterState {
public void removeKeyVersion(String blobKey);
- public void reportError(String stormId, String componentId, String node, Integer port, String error);
+ public void reportError(String stormId, String componentId, String node, Long port, String error);
public List<ErrorInfo> errors(String stormId, String componentId);
[03/27] storm git commit: delete zookeeper.clj
zookeeper_state_factory.clj cluster.clj, but some tests still can't pass
Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/worker.clj b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
index 48934f6..85ed37d 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
@@ -15,11 +15,11 @@
;; limitations under the License.
(ns org.apache.storm.daemon.worker
(:use [org.apache.storm.daemon common])
- (:use [org.apache.storm config log util timer local-state])
+ (:use [org.apache.storm config log util timer local-state converter])
(:require [clj-time.core :as time])
(:require [clj-time.coerce :as coerce])
(:require [org.apache.storm.daemon [executor :as executor]])
- (:require [org.apache.storm [disruptor :as disruptor] [cluster :as cluster]])
+ (:require [org.apache.storm [disruptor :as disruptor]])
(:require [clojure.set :as set])
(:require [org.apache.storm.messaging.loader :as msg-loader])
(:import [java.util.concurrent Executors]
@@ -36,7 +36,7 @@
(:import [org.apache.storm.task WorkerTopologyContext])
(:import [org.apache.storm Constants])
(:import [org.apache.storm.security.auth AuthUtils])
- (:import [org.apache.storm.cluster ClusterStateContext DaemonType])
+ (:import [org.apache.storm.cluster ClusterStateContext DaemonType DistributedClusterState StormZkClusterState])
(:import [javax.security.auth Subject])
(:import [java.security PrivilegedExceptionAction])
(:import [org.apache.logging.log4j LogManager])
@@ -49,7 +49,7 @@
(defn read-worker-executors [storm-conf storm-cluster-state storm-id assignment-id port assignment-versions]
(log-message "Reading Assignments.")
- (let [assignment (:executor->node+port (.assignment-info storm-cluster-state storm-id nil))]
+ (let [assignment (:executor->node+port (clojurify-assignment (.assignmentInfo storm-cluster-state storm-id nil)))]
(doall
(concat
[Constants/SYSTEM_EXECUTOR_ID]
@@ -73,7 +73,7 @@
}]
;; do the zookeeper heartbeat
(try
- (.worker-heartbeat! (:storm-cluster-state worker) (:storm-id worker) (:assignment-id worker) (:port worker) zk-hb)
+ (.workerHeartbeat (:storm-cluster-state worker) (:storm-id worker) (:assignment-id worker) (:port worker) (thriftify-zk-worker-hb zk-hb))
(catch Exception exc
(log-error exc "Worker failed to write heatbeats to ZK or Pacemaker...will retry")))))
@@ -146,7 +146,7 @@
;; update the worker's backpressure flag to zookeeper only when it has changed
(log-debug "BP " @(:backpressure worker) " WAS " prev-backpressure-flag)
(when (not= prev-backpressure-flag @(:backpressure worker))
- (.worker-backpressure! storm-cluster-state storm-id assignment-id port @(:backpressure worker)))
+ (.workerBackpressure storm-cluster-state storm-id assignment-id port @(:backpressure worker)))
))))
(defn- mk-disruptor-backpressure-handler [worker]
@@ -354,10 +354,11 @@
([]
(this (fn [& ignored] (schedule (:refresh-connections-timer worker) 0 this))))
([callback]
- (let [version (.assignment-version storm-cluster-state storm-id callback)
+ (let [version (.assignmentVersion storm-cluster-state storm-id callback)
assignment (if (= version (:version (get @(:assignment-versions worker) storm-id)))
(:data (get @(:assignment-versions worker) storm-id))
- (let [new-assignment (.assignment-info-with-version storm-cluster-state storm-id callback)]
+ (let [java-assignment (.assignmentInfoWithVersion storm-cluster-state storm-id callback)
+ new-assignment {:data (clojurify-assignment (:data java-assignment)) :version version}]
(swap! (:assignment-versions worker) assoc storm-id new-assignment)
(:data new-assignment)))
my-assignment (-> assignment
@@ -403,7 +404,7 @@
([worker]
(refresh-storm-active worker (fn [& ignored] (schedule (:refresh-active-timer worker) 0 (partial refresh-storm-active worker)))))
([worker callback]
- (let [base (.storm-base (:storm-cluster-state worker) (:storm-id worker) callback)]
+ (let [base (clojurify-storm-base (.stormBase (:storm-cluster-state worker) (:storm-id worker) callback))]
(reset!
(:storm-active-atom worker)
(and (= :active (-> base :status :type)) @(:worker-active-flag worker)))
@@ -595,9 +596,9 @@
(let [storm-conf (ConfigUtils/readSupervisorStormConf conf storm-id)
storm-conf (clojurify-structure (ConfigUtils/overrideLoginConfigWithSystemProperty storm-conf))
acls (Utils/getWorkerACL storm-conf)
- cluster-state (cluster/mk-distributed-cluster-state conf :auth-conf storm-conf :acls acls :context (ClusterStateContext. DaemonType/WORKER))
- storm-cluster-state (cluster/mk-storm-cluster-state cluster-state :acls acls)
- initial-credentials (.credentials storm-cluster-state storm-id nil)
+ cluster-state (DistributedClusterState. conf storm-conf acls (ClusterStateContext. DaemonType/WORKER))
+ storm-cluster-state (StormZkClusterState. cluster-state acls (ClusterStateContext.))
+ initial-credentials (clojurify-crdentials (.credentials storm-cluster-state storm-id nil))
auto-creds (AuthUtils/GetAutoCredentials storm-conf)
subject (AuthUtils/populateSubject nil auto-creds initial-credentials)]
(Subject/doAs subject (reify PrivilegedExceptionAction
@@ -644,10 +645,10 @@
_ (if ((:storm-conf worker) TOPOLOGY-BACKPRESSURE-ENABLE)
(.start backpressure-thread))
callback (fn cb [& ignored]
- (let [throttle-on (.topology-backpressure storm-cluster-state storm-id cb)]
+ (let [throttle-on (.topologyBackpressure storm-cluster-state storm-id cb)]
(reset! (:throttle-on worker) throttle-on)))
_ (if ((:storm-conf worker) TOPOLOGY-BACKPRESSURE-ENABLE)
- (.topology-backpressure storm-cluster-state storm-id callback))
+ (.topologyBackpressure storm-cluster-state storm-id callback))
shutdown* (fn []
(log-message "Shutting down worker " storm-id " " assignment-id " " port)
@@ -685,7 +686,7 @@
(log-message "Trigger any worker shutdown hooks")
(run-worker-shutdown-hooks worker)
- (.remove-worker-heartbeat! (:storm-cluster-state worker) storm-id assignment-id port)
+ (.removeWorkerHeartbeat (:storm-cluster-state worker) storm-id assignment-id port)
(log-message "Disconnecting from storm cluster state context")
(.disconnect (:storm-cluster-state worker))
(.close (:cluster-state worker))
@@ -709,29 +710,29 @@
)
credentials (atom initial-credentials)
check-credentials-changed (fn []
- (let [new-creds (.credentials (:storm-cluster-state worker) storm-id nil)]
+ (let [new-creds (clojurify-crdentials (.credentials (:storm-cluster-state worker) storm-id nil))]
(when-not (= new-creds @credentials) ;;This does not have to be atomic, worst case we update when one is not needed
(AuthUtils/updateSubject subject auto-creds new-creds)
(dofor [e @executors] (.credentials-changed e new-creds))
(reset! credentials new-creds))))
check-throttle-changed (fn []
(let [callback (fn cb [& ignored]
- (let [throttle-on (.topology-backpressure (:storm-cluster-state worker) storm-id cb)]
+ (let [throttle-on (.topologyBackpressure (:storm-cluster-state worker) storm-id cb)]
(reset! (:throttle-on worker) throttle-on)))
- new-throttle-on (.topology-backpressure (:storm-cluster-state worker) storm-id callback)]
+ new-throttle-on (.topologyBackpressure (:storm-cluster-state worker) storm-id callback)]
(reset! (:throttle-on worker) new-throttle-on)))
check-log-config-changed (fn []
- (let [log-config (.topology-log-config (:storm-cluster-state worker) storm-id nil)]
+ (let [log-config (.topologyLogConfig (:storm-cluster-state worker) storm-id nil)]
(process-log-config-change latest-log-config original-log-levels log-config)
(establish-log-setting-callback)))]
(reset! original-log-levels (get-logger-levels))
(log-message "Started with log levels: " @original-log-levels)
(defn establish-log-setting-callback []
- (.topology-log-config (:storm-cluster-state worker) storm-id (fn [args] (check-log-config-changed))))
+ (.topologyLogConfig (:storm-cluster-state worker) storm-id (fn [args] (check-log-config-changed))))
(establish-log-setting-callback)
- (.credentials (:storm-cluster-state worker) storm-id (fn [args] (check-credentials-changed)))
+ (clojurify-crdentials (.credentials (:storm-cluster-state worker) storm-id (fn [args] (check-credentials-changed))))
(schedule-recurring (:refresh-credentials-timer worker) 0 (conf TASK-CREDENTIALS-POLL-SECS)
(fn [& args]
(check-credentials-changed)
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj b/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
index cede59e..b367b4b 100644
--- a/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
+++ b/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
@@ -16,27 +16,23 @@
(ns org.apache.storm.pacemaker.pacemaker-state-factory
(:require [org.apache.storm.pacemaker pacemaker]
- [org.apache.storm.cluster-state [zookeeper-state-factory :as zk-factory]]
[org.apache.storm
[config :refer :all]
- [cluster :refer :all]
[log :refer :all]
[util :as util]])
(:import [org.apache.storm.generated
HBExecutionException HBServerMessageType HBMessage
HBMessageData HBPulse]
- [org.apache.storm.cluster_state zookeeper_state_factory]
- [org.apache.storm.cluster ClusterState]
+ [org.apache.storm.cluster ClusterState DistributedClusterState]
[org.apache.storm.pacemaker PacemakerClient])
- (:gen-class
- :implements [org.apache.storm.cluster.ClusterStateFactory]))
+ (:gen-class))
;; So we can mock the client for testing
(defn makeClient [conf]
(PacemakerClient. conf))
(defn makeZKState [conf auth-conf acls context]
- (.mkState (zookeeper_state_factory.) conf auth-conf acls context))
+ (DistributedClusterState. conf auth-conf acls context))
(def max-retries 10)
@@ -47,7 +43,7 @@
(reify
ClusterState
;; Let these pass through to the zk-state. We only want to handle heartbeats.
- (register [this callback] (.register zk-state callback))
+ (register [this callback] (.register zk-state callback)) ; need update callback, have questions?? callback is IFn here
(unregister [this callback] (.unregister zk-state callback))
(set_ephemeral_node [this path data acls] (.set_ephemeral_node zk-state path data acls))
(create_sequential [this path data acls] (.create_sequential zk-state path data acls))
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/stats.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/stats.clj b/storm-core/src/clj/org/apache/storm/stats.clj
index 68b16fd..d6bcdc3 100644
--- a/storm-core/src/clj/org/apache/storm/stats.clj
+++ b/storm-core/src/clj/org/apache/storm/stats.clj
@@ -24,6 +24,7 @@
ExecutorAggregateStats SpecificAggregateStats
SpoutAggregateStats TopologyPageInfo TopologyStats])
(:import [org.apache.storm.utils Utils])
+ (:import [org.apache.storm.cluster StormZkClusterState])
(:import [org.apache.storm.metric.internal MultiCountStatAndMetric MultiLatencyStatAndMetric])
(:use [org.apache.storm log util])
(:use [clojure.math.numeric-tower :only [ceil]]))
@@ -794,7 +795,7 @@
(defn get-last-error
[storm-cluster-state storm-id component-id]
- (if-let [e (.last-error storm-cluster-state storm-id component-id)]
+ (if-let [e (clojurify-error (.lastError storm-cluster-state storm-id component-id))]
(ErrorInfo. (:error e) (:time-secs e))))
(defn component-type
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/testing.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/testing.clj b/storm-core/src/clj/org/apache/storm/testing.clj
index cc78659..eb34d36 100644
--- a/storm-core/src/clj/org/apache/storm/testing.clj
+++ b/storm-core/src/clj/org/apache/storm/testing.clj
@@ -45,10 +45,10 @@
(:import [org.apache.storm.tuple Tuple])
(:import [org.apache.storm.generated StormTopology])
(:import [org.apache.storm.task TopologyContext])
- (:require [org.apache.storm [zookeeper :as zk]])
+ (:import [org.apache.storm.cluster DistributedClusterState ClusterStateContext StormZkClusterState])
(:require [org.apache.storm.messaging.loader :as msg-loader])
(:require [org.apache.storm.daemon.acker :as acker])
- (:use [org.apache.storm cluster util thrift config log local-state]))
+ (:use [org.apache.storm util thrift config log local-state converter]))
(defn feeder-spout
[fields]
@@ -158,8 +158,8 @@
:port-counter port-counter
:daemon-conf daemon-conf
:supervisors (atom [])
- :state (mk-distributed-cluster-state daemon-conf)
- :storm-cluster-state (mk-storm-cluster-state daemon-conf)
+ :state (DistributedClusterState. daemon-conf nil nil (ClusterStateContext.))
+ :storm-cluster-state (StormZkClusterState. daemon-conf nil (ClusterStateContext.))
:tmp-dirs (atom [nimbus-tmp zk-tmp])
:zookeeper (if (not-nil? zk-handle) zk-handle)
:shared-context context
@@ -403,8 +403,8 @@
(select-keys component->tasks component-ids)
component->tasks)
task-ids (apply concat (vals component->tasks))
- assignment (.assignment-info state storm-id nil)
- taskbeats (.taskbeats state storm-id (:task->node+port assignment))
+ assignment (clojurify-assignment (.assignmentInfo state storm-id nil))
+ taskbeats (.taskbeats state storm-id (:task->node+port assignment)) ;hava question?
heartbeats (dofor [id task-ids] (get taskbeats id))
stats (dofor [hb heartbeats] (if hb (stat-key (:stats hb)) 0))]
(reduce + stats)))
@@ -551,7 +551,7 @@
(simulate-wait cluster-map))
(.killTopologyWithOpts (:nimbus cluster-map) storm-name (doto (KillOptions.) (.set_wait_secs 0)))
- (while-timeout timeout-ms (.assignment-info state storm-id nil)
+ (while-timeout timeout-ms (clojurify-assignment (.assignmentInfo state storm-id nil))
(simulate-wait cluster-map))
(when cleanup-state
(doseq [spout (spout-objects spouts)]
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/thrift.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/thrift.clj b/storm-core/src/clj/org/apache/storm/thrift.clj
index b5af521..4dc21f9 100644
--- a/storm-core/src/clj/org/apache/storm/thrift.clj
+++ b/storm-core/src/clj/org/apache/storm/thrift.clj
@@ -30,7 +30,7 @@
(:import [org.apache.storm.topology TopologyBuilder])
(:import [org.apache.storm.clojure RichShellBolt RichShellSpout])
(:import [org.apache.thrift.transport TTransport])
- (:use [org.apache.storm util config log zookeeper]))
+ (:use [org.apache.storm util config log]))
(defn instantiate-java-object
[^JavaObject obj]
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/ui/core.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/ui/core.clj b/storm-core/src/clj/org/apache/storm/ui/core.clj
index f26d998..14313be 100644
--- a/storm-core/src/clj/org/apache/storm/ui/core.clj
+++ b/storm-core/src/clj/org/apache/storm/ui/core.clj
@@ -21,7 +21,7 @@
ring.middleware.multipart-params)
(:use [ring.middleware.json :only [wrap-json-params]])
(:use [hiccup core page-helpers])
- (:use [org.apache.storm config util log stats zookeeper converter])
+ (:use [org.apache.storm config util log stats converter])
(:use [org.apache.storm.ui helpers])
(:use [org.apache.storm.daemon [common :only [ACKER-COMPONENT-ID ACKER-INIT-STREAM-ID ACKER-ACK-STREAM-ID
ACKER-FAIL-STREAM-ID mk-authorization-handler
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/util.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/util.clj b/storm-core/src/clj/org/apache/storm/util.clj
index 23d39f6..165d8ee 100644
--- a/storm-core/src/clj/org/apache/storm/util.clj
+++ b/storm-core/src/clj/org/apache/storm/util.clj
@@ -20,6 +20,7 @@
(:import [java.io FileReader FileNotFoundException])
(:import [java.nio.file Paths])
(:import [org.apache.storm Config])
+ (:import [org.apache.storm.generated ErrorInfo])
(:import [org.apache.storm.utils Time Container ClojureTimerTask Utils
MutableObject MutableInt])
(:import [org.apache.storm.security.auth NimbusPrincipal])
@@ -261,6 +262,16 @@
(instance? Boolean x) (boolean x)
true x))
s))
+; move this func form convert.clj due to cyclic load dependency
+(defn clojurify-error [^ErrorInfo error]
+ (if error
+ {
+ :error (.get_error error)
+ :time-secs (.get_error_time_secs error)
+ :host (.get_host error)
+ :port (.get_port error)
+ }
+ ))
(defmacro with-file-lock
[path & body]
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/zookeeper.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/zookeeper.clj b/storm-core/src/clj/org/apache/storm/zookeeper.clj
deleted file mode 100644
index 413ffd6..0000000
--- a/storm-core/src/clj/org/apache/storm/zookeeper.clj
+++ /dev/null
@@ -1,75 +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.
-
-(ns org.apache.storm.zookeeper
- (:import [org.apache.curator.retry RetryNTimes]
- [org.apache.storm Config])
- (:import [org.apache.curator.framework.api CuratorEvent CuratorEventType CuratorListener UnhandledErrorListener])
- (:import [org.apache.curator.framework.state ConnectionStateListener])
- (:import [org.apache.curator.framework CuratorFramework CuratorFrameworkFactory])
- (:import [org.apache.curator.framework.recipes.leader LeaderLatch LeaderLatch$State Participant LeaderLatchListener])
- (:import [org.apache.zookeeper ZooKeeper Watcher KeeperException$NoNodeException
- ZooDefs ZooDefs$Ids CreateMode WatchedEvent Watcher$Event Watcher$Event$KeeperState
- Watcher$Event$EventType KeeperException$NodeExistsException])
- (:import [org.apache.zookeeper.data Stat])
- (:import [org.apache.zookeeper.server ZooKeeperServer NIOServerCnxnFactory])
- (:import [java.net InetSocketAddress BindException InetAddress])
- (:import [org.apache.storm.nimbus ILeaderElector NimbusInfo])
- (:import [java.io File])
- (:import [java.util List Map])
- (:import [org.apache.storm.zookeeper Zookeeper ZkKeeperStates ZkEventTypes])
- (:import [org.apache.storm.utils Utils ZookeeperAuthInfo])
- (:use [org.apache.storm util log config]))
-
-
-(defn- default-watcher
- [state type path]
- (log-message "Zookeeper state update: " state type path))
-
-(defnk mk-client
- [conf servers port
- :root ""
- :watcher default-watcher
- :auth-conf nil]
- (let [fk (Utils/newCurator conf servers port root (when auth-conf (ZookeeperAuthInfo. auth-conf)))]
- (.. fk
- (getCuratorListenable)
- (addListener
- (reify CuratorListener
- (^void eventReceived [this ^CuratorFramework _fk ^CuratorEvent e]
- (when (= (.getType e) CuratorEventType/WATCHED)
- (let [^WatchedEvent event (.getWatchedEvent e)]
- (watcher (.getState event)
- (.getType event)
- (.getPath event))))))))
- ;; (.. fk
- ;; (getUnhandledErrorListenable)
- ;; (addListener
- ;; (reify UnhandledErrorListener
- ;; (unhandledError [this msg error]
- ;; (if (or (exception-cause? InterruptedException error)
- ;; (exception-cause? java.nio.channels.ClosedByInterruptException error))
- ;; (do (log-warn-error error "Zookeeper exception " msg)
- ;; (let [to-throw (InterruptedException.)]
- ;; (.initCause to-throw error)
- ;; (throw to-throw)
- ;; ))
- ;; (do (log-error error "Unrecoverable Zookeeper error " msg)
- ;; (halt-process! 1 "Unrecoverable Zookeeper error")))
- ;; ))))
- (.start fk)
- fk))
-
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/jvm/org/apache/storm/callback/Callback.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/callback/Callback.java b/storm-core/src/jvm/org/apache/storm/callback/Callback.java
index 29b9761..a37612d 100644
--- a/storm-core/src/jvm/org/apache/storm/callback/Callback.java
+++ b/storm-core/src/jvm/org/apache/storm/callback/Callback.java
@@ -18,6 +18,9 @@
package org.apache.storm.callback;
+import clojure.lang.IFn;
+
+// To remove IFn after porting all callbacks to java
public interface Callback {
public <T> Object execute(T... args);
}
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java b/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java
index 2d6f306..851858f 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java
@@ -27,8 +27,7 @@ import org.apache.zookeeper.ZooDefs;
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Id;
import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+
import java.io.UnsupportedEncodingException;
import java.net.URLEncoder;
@@ -84,10 +83,36 @@ public class Cluster {
PROFILERCONFIG_SUBTREE = ZK_SEPERATOR + PROFILERCONFIG_ROOT;
}
+ // A singleton instance allows us to mock delegated static methods in our
+ // tests by subclassing.
+ private static final Cluster INSTANCE = new Cluster();
+ private static Cluster _instance = INSTANCE;
+
+ /**
+ * Provide an instance of this class for delegates to use. To mock out
+ * delegated methods, provide an instance of a subclass that overrides the
+ * implementation of the delegated method.
+ *
+ * @param u a Zookeeper instance
+ */
+ public static void setInstance(Cluster u) {
+ _instance = u;
+ }
+
+ /**
+ * Resets the singleton instance to the default. This is helpful to reset
+ * the class to its original functionality when mocking is no longer
+ * desired.
+ */
+ public static void resetInstance() {
+ _instance = INSTANCE;
+ }
+
public static List<ACL> mkTopoOnlyAcls(Map topoConf) throws NoSuchAlgorithmException {
- List<ACL> aclList = new ArrayList<>();
+ List<ACL> aclList = null;
String payload = (String)topoConf.get(Config.STORM_ZOOKEEPER_AUTH_PAYLOAD);
if (Utils.isZkAuthenticationConfiguredStormServer(topoConf)){
+ aclList = new ArrayList<>();
ACL acl1 = ZooDefs.Ids.CREATOR_ALL_ACL.get(0);
aclList.add(acl1);
ACL acl2 = new ACL(ZooDefs.Perms.READ, new Id("digest", DigestAuthenticationProvider.generateDigest(payload)));
@@ -182,6 +207,13 @@ public class Cluster {
}
return executorWhb;
}
+
+ public StormClusterState mkStormClusterStateImpl(Object clusterState, List<ACL> acls, ClusterStateContext context) throws Exception{
+ return new StormZkClusterState(clusterState, acls, context);
+ }
+ public static StormClusterState mkStormClusterState(Object clusterState, List<ACL> acls, ClusterStateContext context) throws Exception{
+ return _instance.mkStormClusterStateImpl(clusterState, acls, context);
+ }
// TO be remove
public static <K, V> HashMap<V, List<K>> reverseMap(Map<K, V> map) {
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java
index 51e42ff..e76721b 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java
@@ -46,7 +46,7 @@ public interface ClusterState {
/**
* Registers a callback function that gets called when CuratorEvents happen.
* @param callback is a clojure IFn that accepts the type - translated to
- * clojure keyword as in zookeeper.clj - and the path: (callback type path)
+ * clojure keyword as in zookeeper - and the path: (callback type path)
* @return is an id that can be passed to unregister(...) to unregister the
* callback.
*/
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java
index 3e0beb1..1bd534e 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java
@@ -56,8 +56,7 @@ public class DistributedClusterState implements ClusterState {
public DistributedClusterState(Map<Object, Object> conf, Map authConf, List<ACL> acls, ClusterStateContext context) throws Exception {
this.conf = conf;
this.authConf = authConf;
- if (context.getDaemonType().equals(DaemonType.NIMBUS))
- this.isNimbus = true;
+ if (context.getDaemonType().equals(DaemonType.NIMBUS)) this.isNimbus = true;
// just mkdir STORM_ZOOKEEPER_ROOT dir
CuratorFramework zkTemp = mkZk();
@@ -128,9 +127,9 @@ public class DistributedClusterState implements ClusterState {
}
@Override
- public String register(Callback callback) {
+ public String register( Callback callback) {
String id = UUID.randomUUID().toString();
- this.callbacks.put(id, callback);
+ this.callbacks.put(id,callback);
return id;
}
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
index b3c0f90..ede2ba3 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
@@ -18,7 +18,7 @@
package org.apache.storm.cluster;
import clojure.lang.APersistentMap;
-import org.apache.storm.callback.Callback;
+import clojure.lang.IFn;
import org.apache.storm.generated.*;
import org.apache.storm.nimbus.NimbusInfo;
@@ -27,13 +27,13 @@ import java.util.List;
import java.util.Map;
public interface StormClusterState {
- public List<String> assignments(Callback callback);
+ public List<String> assignments(IFn callback);
- public Assignment assignmentInfo(String stormId, Callback callback);
+ public Assignment assignmentInfo(String stormId, IFn callback);
- public APersistentMap assignmentInfoWithVersion(String stormId, Callback callback);
+ public APersistentMap assignmentInfoWithVersion(String stormId, IFn callback);
- public Integer assignmentVersion(String stormId, Callback callback) throws Exception;
+ public Integer assignmentVersion(String stormId, IFn callback) throws Exception;
// returns key information under /storm/blobstore/key
public List<String> blobstoreInfo(String blobKey);
@@ -46,27 +46,27 @@ public interface StormClusterState {
public List<String> activeStorms();
- public StormBase stormBase(String stormId, Callback callback);
+ public StormBase stormBase(String stormId, IFn callback);
public ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, String node, Long port);
- public List<ProfileRequest> getWorkerProfileRequets(String stormId, NodeInfo nodeInfo, boolean isThrift);
+ public List<ProfileRequest> getWorkerProfileRequests(String stormId, NodeInfo nodeInfo, boolean isThrift);
- public List<ProfileRequest> getTopologyProfileRequets(String stormId, boolean isThrift);
+ public List<ProfileRequest> getTopologyProfileRequests(String stormId, boolean isThrift);
- public void setWorkerProfileRequests(String stormId, ProfileRequest profileRequest);
+ public void setWorkerProfileRequest(String stormId, ProfileRequest profileRequest);
public void deleteTopologyProfileRequests(String stormId, ProfileRequest profileRequest);
public Map<ExecutorInfo, ClusterWorkerHeartbeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort);
- public List<String> supervisors(Callback callback);
+ public List<String> supervisors(IFn callback);
public SupervisorInfo supervisorInfo(String supervisorId); // returns nil if doesn't exist
public void setupHeatbeats(String stormId);
- public void teardownHeatbeats(String stormId);
+ public void teardownHeartbeats(String stormId);
public void teardownTopologyErrors(String stormId);
@@ -76,7 +76,7 @@ public interface StormClusterState {
public void setTopologyLogConfig(String stormId, LogConfig logConfig);
- public LogConfig topologyLogConfig(String stormId, Callback cb);
+ public LogConfig topologyLogConfig(String stormId, IFn cb);
public void workerHeartbeat(String stormId, String node, Long port, ClusterWorkerHeartbeat info);
@@ -86,7 +86,7 @@ public interface StormClusterState {
public void workerBackpressure(String stormId, String node, Long port, boolean on);
- public boolean topologyBackpressure(String stormId, Callback callback);
+ public boolean topologyBackpressure(String stormId, IFn callback);
public void setupBackpressure(String stormId);
@@ -102,11 +102,11 @@ public interface StormClusterState {
// sets up information related to key consisting of nimbus
// host:port and version info of the blob
- public void setupBlobstore(String key, NimbusInfo nimbusInfo, String versionInfo);
+ public void setupBlobstore(String key, NimbusInfo nimbusInfo, Integer versionInfo);
public List<String> activeKeys();
- public List<String> blobstore(Callback callback);
+ public List<String> blobstore(IFn callback);
public void removeStorm(String stormId);
@@ -114,7 +114,7 @@ public interface StormClusterState {
public void removeKeyVersion(String blobKey);
- public void reportError(String stormId, String componentId, String node, Long port, String error);
+ public void reportError(String stormId, String componentId, String node, Integer port, String error);
public List<ErrorInfo> errors(String stormId, String componentId);
@@ -122,7 +122,7 @@ public interface StormClusterState {
public void setCredentials(String stormId, Credentials creds, Map topoConf) throws NoSuchAlgorithmException;
- public Credentials credentials(String stormId, Callback callback);
+ public Credentials credentials(String stormId, IFn callback);
public void disconnect();
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
index 93d29b2..3f32fe1 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
@@ -18,6 +18,7 @@
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;
@@ -47,17 +48,17 @@ public class StormZkClusterState implements StormClusterState {
private ClusterState clusterState;
- private ConcurrentHashMap<String, Callback> assignmentInfoCallback;
- private ConcurrentHashMap<String, Callback> assignmentInfoWithVersionCallback;
- private ConcurrentHashMap<String, Callback> assignmentVersionCallback;
- private AtomicReference<Callback> supervisorsCallback;
+ 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, Callback> backPressureCallback;
- private AtomicReference<Callback> assignmentsCallback;
- private ConcurrentHashMap<String, Callback> stormBaseCallback;
- private AtomicReference<Callback> blobstoreCallback;
- private ConcurrentHashMap<String, Callback> credentialsCallback;
- private ConcurrentHashMap<String, Callback> logConfigCallback;
+ 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;
@@ -102,7 +103,7 @@ public class StormZkClusterState implements StormClusterState {
if (size >= 1) {
String params = null;
String root = toks.get(0);
- Callback fn = null;
+ IFn fn = null;
if (root.equals(Cluster.ASSIGNMENTS_ROOT)) {
if (size == 1) {
// set null and get the old value
@@ -145,18 +146,18 @@ public class StormZkClusterState implements StormClusterState {
}
- protected void issueCallback(AtomicReference<Callback> cb) {
- Callback callback = cb.getAndSet(null);
- callback.execute();
+ protected void issueCallback(AtomicReference<IFn> cb) {
+ IFn callback = cb.getAndSet(null);
+ callback.invoke();
}
- protected void issueMapCallback(ConcurrentHashMap<String, Callback> callbackConcurrentHashMap, String key) {
- Callback callback = callbackConcurrentHashMap.remove(key);
- callback.execute();
+ protected void issueMapCallback(ConcurrentHashMap<String, IFn> callbackConcurrentHashMap, String key) {
+ IFn callback = callbackConcurrentHashMap.remove(key);
+ callback.invoke();
}
@Override
- public List<String> assignments(Callback callback) {
+ public List<String> assignments(IFn callback) {
if (callback != null) {
assignmentsCallback.set(callback);
}
@@ -164,7 +165,7 @@ public class StormZkClusterState implements StormClusterState {
}
@Override
- public Assignment assignmentInfo(String stormId, Callback callback) {
+ public Assignment assignmentInfo(String stormId, IFn callback) {
if (callback != null) {
assignmentInfoCallback.put(stormId, callback);
}
@@ -173,7 +174,7 @@ public class StormZkClusterState implements StormClusterState {
}
@Override
- public APersistentMap assignmentInfoWithVersion(String stormId, Callback callback) {
+ public APersistentMap assignmentInfoWithVersion(String stormId, IFn callback) {
if (callback != null) {
assignmentInfoWithVersionCallback.put(stormId, callback);
}
@@ -185,7 +186,7 @@ public class StormZkClusterState implements StormClusterState {
}
@Override
- public Integer assignmentVersion(String stormId, Callback callback) throws Exception {
+ public Integer assignmentVersion(String stormId, IFn callback) throws Exception {
if (callback != null) {
assignmentVersionCallback.put(stormId, callback);
}
@@ -237,7 +238,7 @@ public class StormZkClusterState implements StormClusterState {
}
@Override
- public StormBase stormBase(String stormId, Callback callback) {
+ public StormBase stormBase(String stormId, IFn callback) {
if (callback != null) {
stormBaseCallback.put(stormId, callback);
}
@@ -254,9 +255,9 @@ public class StormZkClusterState implements StormClusterState {
}
@Override
- public List<ProfileRequest> getWorkerProfileRequets(String stormId, NodeInfo nodeInfo, boolean isThrift) {
+ public List<ProfileRequest> getWorkerProfileRequests(String stormId, NodeInfo nodeInfo, boolean isThrift) {
List<ProfileRequest> requests = new ArrayList<>();
- List<ProfileRequest> profileRequests = getTopologyProfileRequets(stormId, isThrift);
+ List<ProfileRequest> profileRequests = getTopologyProfileRequests(stormId, isThrift);
for (ProfileRequest profileRequest : profileRequests) {
NodeInfo nodeInfo1 = profileRequest.get_nodeInfo();
if (nodeInfo1.equals(nodeInfo))
@@ -266,7 +267,7 @@ public class StormZkClusterState implements StormClusterState {
}
@Override
- public List<ProfileRequest> getTopologyProfileRequets(String stormId, boolean isThrift) {
+ public List<ProfileRequest> getTopologyProfileRequests(String stormId, boolean isThrift) {
List<ProfileRequest> profileRequests = new ArrayList<>();
String path = Cluster.profilerConfigPath(stormId);
if (clusterState.node_exists(path, false)) {
@@ -283,7 +284,7 @@ public class StormZkClusterState implements StormClusterState {
}
@Override
- public void setWorkerProfileRequests(String stormId, ProfileRequest profileRequest) {
+ public void setWorkerProfileRequest(String stormId, ProfileRequest profileRequest) {
ProfileAction profileAction = profileRequest.get_action();
String host = profileRequest.get_nodeInfo().get_node();
Long port = profileRequest.get_nodeInfo().get_port_iterator().next();
@@ -300,11 +301,18 @@ public class StormZkClusterState implements StormClusterState {
clusterState.delete_node(path);
}
+ // need to take executor->node+port in explicitly so that we don't run into a situation where a
+ // long dead worker with a skewed clock overrides all the timestamps. By only checking heartbeats
+ // with an assigned node+port, and only reading executors from that heartbeat that are actually assigned,
+ // we avoid situations like that
@Override
public Map<ExecutorInfo, ClusterWorkerHeartbeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort) {
Map<ExecutorInfo, ClusterWorkerHeartbeat> executorWhbs = new HashMap<>();
+ LOG.info(executorNodePort.toString());
Map<NodeInfo, List<List<Long>>> nodePortExecutors = Cluster.reverseMap(executorNodePort);
+ LOG.info(nodePortExecutors.toString());
+
for (Map.Entry<NodeInfo, List<List<Long>>> entry : nodePortExecutors.entrySet()) {
String node = entry.getKey().get_node();
@@ -320,7 +328,7 @@ public class StormZkClusterState implements StormClusterState {
}
@Override
- public List<String> supervisors(Callback callback) {
+ public List<String> supervisors(IFn callback) {
if (callback != null) {
supervisorsCallback.set(callback);
}
@@ -339,7 +347,7 @@ public class StormZkClusterState implements StormClusterState {
}
@Override
- public void teardownHeatbeats(String stormId) {
+ public void teardownHeartbeats(String stormId) {
try {
clusterState.delete_worker_hb(Cluster.workerbeatStormRoot(stormId));
} catch (Exception e) {
@@ -382,7 +390,7 @@ public class StormZkClusterState implements StormClusterState {
}
@Override
- public LogConfig topologyLogConfig(String stormId, Callback cb) {
+ public LogConfig topologyLogConfig(String stormId, IFn cb) {
String path = Cluster.logConfigPath(stormId);
return Cluster.maybeDeserialize(clusterState.get_data(path, cb != null), LogConfig.class);
}
@@ -426,7 +434,7 @@ public class StormZkClusterState implements StormClusterState {
// if the backpresure/storm-id dir is empty, this topology has throttle-on, otherwise not.
@Override
- public boolean topologyBackpressure(String stormId, Callback callback) {
+ public boolean topologyBackpressure(String stormId, IFn callback) {
if (callback != null) {
backPressureCallback.put(stormId, callback);
}
@@ -458,26 +466,27 @@ public class StormZkClusterState implements StormClusterState {
StormBase stormBase = stormBase(stormId, null);
if (stormBase.get_component_executors() != null) {
+
+ Map<String, Integer> newComponentExecutors = new HashMap<>();
Map<String, Integer> componentExecutors = newElems.get_component_executors();
- if (componentExecutors == null) {
- componentExecutors = new HashMap<>();
+ //componentExecutors maybe be APersistentMap, which don't support put
+ for (Map.Entry<String, Integer> entry : componentExecutors.entrySet()) {
+ newComponentExecutors.put(entry.getKey(), entry.getValue());
}
for (Map.Entry<String, Integer> entry : stormBase.get_component_executors().entrySet()) {
if (!componentExecutors.containsKey(entry.getKey())) {
- componentExecutors.put(entry.getKey(), entry.getValue());
+ newComponentExecutors.put(entry.getKey(), entry.getValue());
}
}
- if (componentExecutors.size() > 0)
- newElems.set_component_executors(componentExecutors);
+ if (newComponentExecutors.size() > 0)
+ newElems.set_component_executors(newComponentExecutors);
}
Map<String, DebugOptions> ComponentDebug = new HashMap<>();
Map<String, DebugOptions> oldComponentDebug = stormBase.get_component_debug();
- if (oldComponentDebug == null)
- oldComponentDebug = new HashMap<>();
+
Map<String, DebugOptions> newComponentDebug = newElems.get_component_debug();
- if (newComponentDebug == null)
- newComponentDebug = new HashMap<>();
+
Set<String> debugOptionsKeys = oldComponentDebug.keySet();
debugOptionsKeys.addAll(newComponentDebug.keySet());
for (String key : debugOptionsKeys) {
@@ -499,7 +508,17 @@ public class StormZkClusterState implements StormClusterState {
if (ComponentDebug.size() > 0) {
newElems.set_component_debug(ComponentDebug);
}
- // only merge some parameters which are optional
+
+
+ if (StringUtils.isBlank(newElems.get_name())) {
+ newElems.set_name(stormBase.get_name());
+ }
+ if (newElems.get_status() == null){
+ newElems.set_status(stormBase.get_status());
+ }
+ if (newElems.get_num_workers() == 0){
+ newElems.set_num_workers(stormBase.get_num_workers());
+ }
if (newElems.get_launch_time_secs() == 0) {
newElems.set_launch_time_secs(stormBase.get_launch_time_secs());
}
@@ -526,8 +545,8 @@ public class StormZkClusterState implements StormClusterState {
}
@Override
- public void setupBlobstore(String key, NimbusInfo nimbusInfo, String versionInfo) {
- String path = Cluster.blobstorePath(key) + Cluster.ZK_SEPERATOR + nimbusInfo.toHostPortString() + "_" + versionInfo;
+ public void setupBlobstore(String key, NimbusInfo nimbusInfo, Integer versionInfo) {
+ String path = Cluster.blobstorePath(key) + Cluster.ZK_SEPERATOR + nimbusInfo.toHostPortString() + "-" + versionInfo;
LOG.info("set-path: {}", path);
clusterState.mkdirs(Cluster.blobstorePath(key), acls);
clusterState.delete_node_blobstore(Cluster.blobstorePath(key), nimbusInfo.toHostPortString());
@@ -541,7 +560,7 @@ public class StormZkClusterState implements StormClusterState {
// blobstore state
@Override
- public List<String> blobstore(Callback callback) {
+ public List<String> blobstore(IFn callback) {
if (callback != null) {
blobstoreCallback.set(callback);
}
@@ -571,7 +590,7 @@ public class StormZkClusterState implements StormClusterState {
}
@Override
- public void reportError(String stormId, String componentId, String node, Long port, String error) {
+ public void reportError(String stormId, String componentId, String node, Integer port, String error) {
try {
String path = Cluster.errorPath(stormId, componentId);
@@ -644,7 +663,7 @@ public class StormZkClusterState implements StormClusterState {
}
@Override
- public Credentials credentials(String stormId, Callback callback) {
+ public Credentials credentials(String stormId, IFn callback) {
if (callback != null) {
credentialsCallback.put(stormId, callback);
}
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedCluster.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedCluster.java b/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedCluster.java
new file mode 100644
index 0000000..5d67a54
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedCluster.java
@@ -0,0 +1,31 @@
+/**
+ * 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.testing.staticmocking;
+
+import org.apache.storm.cluster.Cluster;
+
+public class MockedCluster implements AutoCloseable {
+
+ public MockedCluster(Cluster inst) {
+ Cluster.setInstance(inst);
+ }
+
+ @Override
+ public void close() throws Exception {
+ Cluster.resetInstance();
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/integration/org/apache/storm/integration_test.clj b/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
index cd2bc26..d374511 100644
--- a/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
+++ b/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
@@ -21,7 +21,8 @@
(:import [org.apache.storm.testing TestWordCounter TestWordSpout TestGlobalCount
TestAggregatesCounter TestConfBolt AckFailMapTracker AckTracker TestPlannerSpout])
(:import [org.apache.storm.tuple Fields])
- (:use [org.apache.storm testing config clojure util])
+ (:import [org.apache.storm.cluster StormZkClusterState])
+ (:use [org.apache.storm testing config clojure util converter])
(:use [org.apache.storm.daemon common])
(:require [org.apache.storm [thrift :as thrift]]))
@@ -575,34 +576,34 @@
(:topology tracked))
_ (advance-cluster-time cluster 11)
storm-id (get-storm-id state "test-errors")
- errors-count (fn [] (count (.errors state storm-id "2")))]
+ errors-count (fn [] (count (clojurify-error (.errors state storm-id "2"))))]
- (is (nil? (.last-error state storm-id "2")))
+ (is (nil? (clojurify-error (.lastError state storm-id "2"))))
;; so it launches the topology
(advance-cluster-time cluster 2)
(.feed feeder [6])
(tracked-wait tracked 1)
(is (= 4 (errors-count)))
- (is (.last-error state storm-id "2"))
+ (is (clojurify-error (.lastError state storm-id "2")))
(advance-time-secs! 5)
(.feed feeder [2])
(tracked-wait tracked 1)
(is (= 4 (errors-count)))
- (is (.last-error state storm-id "2"))
+ (is (clojurify-error (.lastError state storm-id "2")))
(advance-time-secs! 6)
(.feed feeder [2])
(tracked-wait tracked 1)
(is (= 6 (errors-count)))
- (is (.last-error state storm-id "2"))
+ (is (clojurify-error (.lastError state storm-id "2")))
(advance-time-secs! 6)
(.feed feeder [3])
(tracked-wait tracked 1)
(is (= 8 (errors-count)))
- (is (.last-error state storm-id "2"))))))
+ (is (clojurify-error (.lastError state storm-id "2")))))))
(deftest test-acking-branching-complex
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/test/clj/org/apache/storm/cluster_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/cluster_test.clj b/storm-core/test/clj/org/apache/storm/cluster_test.clj
index ffd913e..d0b9882 100644
--- a/storm-core/test/clj/org/apache/storm/cluster_test.clj
+++ b/storm-core/test/clj/org/apache/storm/cluster_test.clj
@@ -23,14 +23,13 @@
(:import [org.mockito.exceptions.base MockitoAssertionError])
(:import [org.apache.curator.framework CuratorFramework CuratorFrameworkFactory CuratorFrameworkFactory$Builder])
(:import [org.apache.storm.utils Utils TestUtils ZookeeperAuthInfo ConfigUtils])
- (:import [org.apache.storm.cluster ClusterState])
+ (:import [org.apache.storm.cluster ClusterState DistributedClusterState ClusterStateContext StormZkClusterState])
(:import [org.apache.storm.zookeeper Zookeeper])
(:import [org.apache.storm.testing.staticmocking MockedZookeeper])
- (:require [org.apache.storm [zookeeper :as zk]])
(:require [conjure.core])
(:use [conjure core])
(:use [clojure test])
- (:use [org.apache.storm cluster config util testing thrift log]))
+ (:use [org.apache.storm config util testing thrift log converter]))
(defn mk-config [zk-port]
(merge (clojurify-structure (ConfigUtils/readStormConfig))
@@ -39,13 +38,13 @@
(defn mk-state
([zk-port] (let [conf (mk-config zk-port)]
- (mk-distributed-cluster-state conf :auth-conf conf)))
+ (DistributedClusterState. conf conf nil (ClusterStateContext.))))
([zk-port cb]
(let [ret (mk-state zk-port)]
(.register ret cb)
ret )))
-(defn mk-storm-state [zk-port] (mk-storm-cluster-state (mk-config zk-port)))
+(defn mk-storm-state [zk-port] (StormZkClusterState. (mk-config zk-port) nil (ClusterStateContext.)))
(deftest test-basics
(with-inprocess-zookeeper zk-port
@@ -182,48 +181,48 @@
base1 (StormBase. "/tmp/storm1" 1 {:type :active} 2 {} "" nil nil {})
base2 (StormBase. "/tmp/storm2" 2 {:type :active} 2 {} "" nil nil {})]
(is (= [] (.assignments state nil)))
- (.set-assignment! state "storm1" assignment1)
- (is (= assignment1 (.assignment-info state "storm1" nil)))
- (is (= nil (.assignment-info state "storm3" nil)))
- (.set-assignment! state "storm1" assignment2)
- (.set-assignment! state "storm3" assignment1)
+ (.setAssignment state "storm1" (thriftify-assignment assignment1))
+ (is (= assignment1 (clojurify-assignment (.assignmentInfo state "storm1" nil))))
+ (is (= nil (clojurify-assignment (.assignmentInfo state "storm3" nil))))
+ (.setAssignment state "storm1" (thriftify-assignment assignment2))
+ (.setAssignment state "storm3" (thriftify-assignment assignment1))
(is (= #{"storm1" "storm3"} (set (.assignments state nil))))
- (is (= assignment2 (.assignment-info state "storm1" nil)))
- (is (= assignment1 (.assignment-info state "storm3" nil)))
+ (is (= assignment2 (clojurify-assignment (.assignmentInfo state "storm1" nil))))
+ (is (= assignment1 (clojurify-assignment (.assignmentInfo state "storm3" nil))))
(is (= [] (.active-storms state)))
- (.activate-storm! state "storm1" base1)
+ (.activateStorm state "storm1" (thriftify-storm-base base1))
(is (= ["storm1"] (.active-storms state)))
- (is (= base1 (.storm-base state "storm1" nil)))
- (is (= nil (.storm-base state "storm2" nil)))
- (.activate-storm! state "storm2" base2)
- (is (= base1 (.storm-base state "storm1" nil)))
- (is (= base2 (.storm-base state "storm2" nil)))
+ (is (= base1 (clojurify-storm-base (.stormBase state "storm1" nil))))
+ (is (= nil (clojurify-storm-base (.stormBase state "storm2" nil))))
+ (.activateStorm state "storm2" (thriftify-storm-base base2))
+ (is (= base1 (clojurify-storm-base (.stormBase state "storm1" nil))))
+ (is (= base2 (clojurify-storm-base (.stormBase state "storm2" nil))))
(is (= #{"storm1" "storm2"} (set (.active-storms state))))
- (.remove-storm-base! state "storm1")
- (is (= base2 (.storm-base state "storm2" nil)))
+ (.removeStormBase state "storm1")
+ (is (= base2 (clojurify-storm-base (.stormBase state "storm2" nil))))
(is (= #{"storm2"} (set (.active-storms state))))
- (is (nil? (.credentials state "storm1" nil)))
- (.set-credentials! state "storm1" {"a" "a"} {})
- (is (= {"a" "a"} (.credentials state "storm1" nil)))
- (.set-credentials! state "storm1" {"b" "b"} {})
- (is (= {"b" "b"} (.credentials state "storm1" nil)))
+ (is (nil? (clojurify-crdentials (.credentials state "storm1" nil))))
+ (.setCredentials! state "storm1" (thriftify-credentials {"a" "a"}) {})
+ (is (= {"a" "a"} (clojurify-crdentials (.credentials state "storm1" nil))))
+ (.setCredentials state "storm1" (thriftify-credentials {"b" "b"}) {})
+ (is (= {"b" "b"} (clojurify-crdentials (.credentials state "storm1" 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 (= [] (.blobstoreInfo state nil)))
+ (.setupBlobstore state "key1" nimbusInfo1 "1")
+ (is (= ["key1"] (.blobstoreInfo state nil)))
+ (is (= [(str (.toHostPortString nimbusInfo1) "-1")] (.blobstoreInfo state "key1")))
+ (.setupBlobstore 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)))
+ (str (.toHostPortString nimbusInfo2) "-1")} (set (.blobstoreInfo state "key1"))))
+ (.removeBlobstoreKey state "key1")
+ (is (= [] (.blobstoreInfo state nil)))
(is (= [] (.nimbuses state)))
- (.add-nimbus-host! state "nimbus1:port" nimbusSummary1)
+ (.addNimbusHost state "nimbus1:port" nimbusSummary1)
(is (= [nimbusSummary1] (.nimbuses state)))
- (.add-nimbus-host! state "nimbus2:port" nimbusSummary2)
+ (.addNimbusHost state "nimbus2:port" nimbusSummary2)
(is (= #{nimbusSummary1 nimbusSummary2} (set (.nimbuses state))))
;; TODO add tests for task info and task heartbeat setting and getting
@@ -231,7 +230,7 @@
)))
(defn- validate-errors! [state storm-id component errors-list]
- (let [errors (.errors state storm-id component)]
+ (let [errors (clojurify-error (.errors state storm-id component))]
;;(println errors)
(is (= (count errors) (count errors-list)))
(doseq [[error target] (map vector errors errors-list)]
@@ -245,17 +244,17 @@
(with-inprocess-zookeeper zk-port
(with-simulated-time
(let [state (mk-storm-state zk-port)]
- (.report-error state "a" "1" (local-hostname) 6700 (RuntimeException.))
+ (.reportError state "a" "1" (local-hostname) 6700 (stringify-error (RuntimeException.)))
(validate-errors! state "a" "1" ["RuntimeException"])
(advance-time-secs! 1)
- (.report-error state "a" "1" (local-hostname) 6700 (IllegalArgumentException.))
+ (.reportError state "a" "1" (local-hostname) 6700 (stringify-error (IllegalArgumentException.)))
(validate-errors! state "a" "1" ["IllegalArgumentException" "RuntimeException"])
(doseq [i (range 10)]
- (.report-error state "a" "2" (local-hostname) 6700 (RuntimeException.))
+ (.reportError state "a" "2" (local-hostname) 6700 (stringify-error (RuntimeException.)))
(advance-time-secs! 2))
(validate-errors! state "a" "2" (repeat 10 "RuntimeException"))
(doseq [i (range 5)]
- (.report-error state "a" "2" (local-hostname) 6700 (IllegalArgumentException.))
+ (.reportError state "a" "2" (local-hostname) 6700 (stringify-error (IllegalArgumentException.)))
(advance-time-secs! 2))
(validate-errors! state "a" "2" (concat (repeat 5 "IllegalArgumentException")
(repeat 5 "RuntimeException")
@@ -271,10 +270,10 @@
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)]
(is (= [] (.supervisors state1 nil)))
- (.supervisor-heartbeat! state2 "2" supervisor-info2)
- (.supervisor-heartbeat! state1 "1" supervisor-info1)
- (is (= supervisor-info2 (.supervisor-info state1 "2")))
- (is (= supervisor-info1 (.supervisor-info state1 "1")))
+ (.supervisorHeartbeat state2 "2" (thriftify-supervisor-info supervisor-info2))
+ (.supervisorHeartbeat state1 "1" (thriftify-supervisor-info supervisor-info1))
+ (is (= supervisor-info2 (clojurify-supervisor-info (.supervisorInfo state1 "2"))))
+ (is (= supervisor-info1 (clojurify-supervisor-info (.supervisorInfo state1 "1"))))
(is (= #{"1" "2"} (set (.supervisors state1 nil))))
(is (= #{"1" "2"} (set (.supervisors state2 nil))))
(.disconnect state2)
@@ -313,12 +312,10 @@
(let [zk-mock (Mockito/mock Zookeeper)]
;; No need for when clauses because we just want to return nil
(with-open [_ (MockedZookeeper. zk-mock)]
- (stubbing [zk/mk-client (reify CuratorFramework (^void close [this] nil))]
- (mk-distributed-cluster-state {})
- (.mkdirsImpl (Mockito/verify zk-mock (Mockito/times 1)) (Mockito/any) (Mockito/anyString) (Mockito/eq nil)))))
- (stubbing [mk-distributed-cluster-state (reify ClusterState
- (register [this callback] nil)
- (mkdirs [this path acls] nil))]
- (mk-storm-cluster-state {})
- (verify-call-times-for mk-distributed-cluster-state 1)
- (verify-first-call-args-for-indices mk-distributed-cluster-state [4] nil))))
+ (. (Mockito/when (Mockito/mock Zookeeper)) (thenReturn (reify CuratorFramework (^void close [this] nil))))
+ (. (Mockito/when (Mockito/mock DistributedClusterState)) (thenReturn {}))
+ (. (Mockito/when (Mockito/mock StormZkClusterState)) (thenReturn (reify ClusterState
+ (register [this callback] nil)
+ (mkdirs [this path acls] nil))))
+ (.mkdirsImpl (Mockito/verify zk-mock (Mockito/times 1)) (Mockito/any) (Mockito/anyString) (Mockito/eq nil))))))
+
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/test/clj/org/apache/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/nimbus_test.clj b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
index 19c6f59..d4402fb 100644
--- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj
+++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
@@ -23,8 +23,10 @@
[org.apache.storm.nimbus InMemoryTopologyActionNotifier])
(:import [org.apache.storm.testing.staticmocking MockedZookeeper])
(:import [org.apache.storm.scheduler INimbus])
+ (:import [org.mockito Mockito])
+ (:import [org.mockito.exceptions.base MockitoAssertionError])
(:import [org.apache.storm.nimbus ILeaderElector NimbusInfo])
- (:import [org.apache.storm.testing.staticmocking MockedConfigUtils])
+ (:import [org.apache.storm.testing.staticmocking MockedConfigUtils MockedCluster])
(:import [org.apache.storm.generated Credentials NotAliveException SubmitOptions
TopologyInitialStatus TopologyStatus AlreadyAliveException KillOptions RebalanceOptions
InvalidTopologyException AuthorizationException
@@ -34,12 +36,12 @@
(:import [org.apache.storm.utils Time Utils ConfigUtils])
(:import [org.apache.storm.zookeeper Zookeeper])
(:import [org.apache.commons.io FileUtils])
- (:use [org.apache.storm testing MockAutoCred util config log timer zookeeper])
+ (:import [org.apache.storm.cluster StormZkClusterState ClusterStateContext Cluster])
+ (:use [org.apache.storm testing MockAutoCred util config log timer converter])
(:use [org.apache.storm.daemon common])
(:require [conjure.core])
(:require [org.apache.storm
- [thrift :as thrift]
- [cluster :as cluster]])
+ [thrift :as thrift]])
(:use [conjure core]))
(defn storm-component->task-info [cluster storm-name]
@@ -51,7 +53,7 @@
(defn getCredentials [cluster storm-name]
(let [storm-id (get-storm-id (:storm-cluster-state cluster) storm-name)]
- (.credentials (:storm-cluster-state cluster) storm-id nil)))
+ (clojurify-crdentials (.credentials (:storm-cluster-state cluster) storm-id nil))))
(defn storm-component->executor-info [cluster storm-name]
(let [storm-id (get-storm-id (:storm-cluster-state cluster) storm-name)
@@ -61,7 +63,7 @@
task->component (storm-task-info topology storm-conf)
state (:storm-cluster-state cluster)
get-component (comp task->component first)]
- (->> (.assignment-info state storm-id nil)
+ (->> (clojurify-assignment (.assignmentInfo state storm-id nil))
:executor->node+port
keys
(map (fn [e] {e (get-component e)}))
@@ -70,13 +72,13 @@
(defn storm-num-workers [state storm-name]
(let [storm-id (get-storm-id state storm-name)
- assignment (.assignment-info state storm-id nil)]
+ assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
(count (reverse-map (:executor->node+port assignment)))
))
(defn topology-nodes [state storm-name]
(let [storm-id (get-storm-id state storm-name)
- assignment (.assignment-info state storm-id nil)]
+ assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
(->> assignment
:executor->node+port
vals
@@ -86,7 +88,7 @@
(defn topology-slots [state storm-name]
(let [storm-id (get-storm-id state storm-name)
- assignment (.assignment-info state storm-id nil)]
+ assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
(->> assignment
:executor->node+port
vals
@@ -95,7 +97,7 @@
(defn topology-node-distribution [state storm-name]
(let [storm-id (get-storm-id state storm-name)
- assignment (.assignment-info state storm-id nil)]
+ assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
(->> assignment
:executor->node+port
vals
@@ -111,28 +113,28 @@
(defn executor-assignment [cluster storm-id executor-id]
(let [state (:storm-cluster-state cluster)
- assignment (.assignment-info state storm-id nil)]
+ assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
((:executor->node+port assignment) executor-id)
))
(defn executor-start-times [cluster storm-id]
(let [state (:storm-cluster-state cluster)
- assignment (.assignment-info state storm-id nil)]
+ assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
(:executor->start-time-secs assignment)))
(defn do-executor-heartbeat [cluster storm-id executor]
(let [state (:storm-cluster-state cluster)
- executor->node+port (:executor->node+port (.assignment-info state storm-id nil))
+ executor->node+port (:executor->node+port (clojurify-assignment (.assignmentInfo state storm-id nil)))
[node port] (get executor->node+port executor)
- curr-beat (.get-worker-heartbeat state storm-id node port)
+ curr-beat (clojurify-zk-worker-hb (.getworkerHeartbeat state storm-id node port))
stats (:executor-stats curr-beat)]
- (.worker-heartbeat! state storm-id node port
- {:storm-id storm-id :time-secs (current-time-secs) :uptime 10 :executor-stats (merge stats {executor (stats/render-stats! (stats/mk-bolt-stats 20))})}
+ (.workerHeartbeat state storm-id node port
+ (thriftify-zk-worker-hb {:storm-id storm-id :time-secs (current-time-secs) :uptime 10 :executor-stats (merge stats {executor (stats/render-stats! (stats/mk-bolt-stats 20))})})
)))
(defn slot-assignments [cluster storm-id]
(let [state (:storm-cluster-state cluster)
- assignment (.assignment-info state storm-id nil)]
+ assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
(reverse-map (:executor->node+port assignment))
))
@@ -144,7 +146,7 @@
(defn topology-executors [cluster storm-id]
(let [state (:storm-cluster-state cluster)
- assignment (.assignment-info state storm-id nil)]
+ assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
(keys (:executor->node+port assignment))
))
@@ -162,7 +164,7 @@
(let [state (:storm-cluster-state cluster)
storm-id (get-storm-id state storm-name)
task-ids (task-ids cluster storm-id)
- assignment (.assignment-info state storm-id nil)
+ assignment (clojurify-assignment (.assignmentInfo state storm-id nil))
executor->node+port (:executor->node+port assignment)
task->node+port (to-task->node+port executor->node+port)
assigned-task-ids (mapcat executor-id->tasks (keys executor->node+port))
@@ -419,54 +421,54 @@
(submit-local-topology (:nimbus cluster) "test" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 20, LOGS-USERS ["alice", (System/getProperty "user.name")]} topology)
(bind storm-id (get-storm-id state "test"))
(advance-cluster-time cluster 5)
- (is (not-nil? (.storm-base state storm-id nil)))
- (is (not-nil? (.assignment-info state storm-id nil)))
+ (is (not-nil? (clojurify-storm-base (.stormBase state storm-id nil))))
+ (is (not-nil? (clojurify-assignment (.assignmentInfo state storm-id nil))))
(.killTopology (:nimbus cluster) "test")
;; check that storm is deactivated but alive
- (is (= :killed (-> (.storm-base state storm-id nil) :status :type)))
- (is (not-nil? (.assignment-info state storm-id nil)))
+ (is (= :killed (-> (clojurify-storm-base (.stormBase state storm-id nil)) :status :type)))
+ (is (not-nil? (clojurify-assignment (.assignmentInfo state storm-id nil))))
(advance-cluster-time cluster 35)
;; kill topology read on group
(submit-local-topology (:nimbus cluster) "killgrouptest" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 20, LOGS-GROUPS ["alice-group"]} topology)
(bind storm-id-killgroup (get-storm-id state "killgrouptest"))
(advance-cluster-time cluster 5)
- (is (not-nil? (.storm-base state storm-id-killgroup nil)))
- (is (not-nil? (.assignment-info state storm-id-killgroup nil)))
+ (is (not-nil? (clojurify-storm-base (.stormBase state storm-id-killgroup nil))))
+ (is (not-nil? (clojurify-assignment (.assignmentInfo state storm-id-killgroup nil))))
(.killTopology (:nimbus cluster) "killgrouptest")
;; check that storm is deactivated but alive
- (is (= :killed (-> (.storm-base state storm-id-killgroup nil) :status :type)))
- (is (not-nil? (.assignment-info state storm-id-killgroup nil)))
+ (is (= :killed (-> (clojurify-storm-base (.stormBase state storm-id-killgroup nil)) :status :type)))
+ (is (not-nil? (clojurify-assignment (.assignmentInfo state storm-id-killgroup nil))))
(advance-cluster-time cluster 35)
;; kill topology can't read
(submit-local-topology (:nimbus cluster) "killnoreadtest" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 20} topology)
(bind storm-id-killnoread (get-storm-id state "killnoreadtest"))
(advance-cluster-time cluster 5)
- (is (not-nil? (.storm-base state storm-id-killnoread nil)))
- (is (not-nil? (.assignment-info state storm-id-killnoread nil)))
+ (is (not-nil? (clojurify-storm-base (.stormBase state storm-id-killnoread nil))))
+ (is (not-nil? (clojurify-assignment (.assignmentInfo state storm-id-killnoread nil))))
(.killTopology (:nimbus cluster) "killnoreadtest")
;; check that storm is deactivated but alive
- (is (= :killed (-> (.storm-base state storm-id-killnoread nil) :status :type)))
- (is (not-nil? (.assignment-info state storm-id-killnoread nil)))
+ (is (= :killed (-> (clojurify-storm-base (.stormBase state storm-id-killnoread nil)) :status :type)))
+ (is (not-nil? (clojurify-assignment (.assignmentInfo state storm-id-killnoread nil))))
(advance-cluster-time cluster 35)
;; active topology can read
(submit-local-topology (:nimbus cluster) "2test" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 10, LOGS-USERS ["alice", (System/getProperty "user.name")]} topology)
(advance-cluster-time cluster 11)
(bind storm-id2 (get-storm-id state "2test"))
- (is (not-nil? (.storm-base state storm-id2 nil)))
- (is (not-nil? (.assignment-info state storm-id2 nil)))
+ (is (not-nil? (clojurify-storm-base (.stormBase state storm-id2 nil))))
+ (is (not-nil? (clojurify-assignment (.assignmentInfo state storm-id2 nil))))
;; active topology can not read
(submit-local-topology (:nimbus cluster) "testnoread" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 10, LOGS-USERS ["alice"]} topology)
(advance-cluster-time cluster 11)
(bind storm-id3 (get-storm-id state "testnoread"))
- (is (not-nil? (.storm-base state storm-id3 nil)))
- (is (not-nil? (.assignment-info state storm-id3 nil)))
+ (is (not-nil? (clojurify-storm-base (.stormBase state storm-id3 nil))))
+ (is (not-nil? (clojurify-assignment (.assignmentInfo state storm-id3 nil))))
;; active topology can read based on group
(submit-local-topology (:nimbus cluster) "testreadgroup" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 10, LOGS-GROUPS ["alice-group"]} topology)
(advance-cluster-time cluster 11)
(bind storm-id4 (get-storm-id state "testreadgroup"))
- (is (not-nil? (.storm-base state storm-id4 nil)))
- (is (not-nil? (.assignment-info state storm-id4 nil)))
+ (is (not-nil? (clojurify-storm-base (.stormBase state storm-id4 nil))))
+ (is (not-nil? (clojurify-assignment (.assignmentInfo state storm-id4 nil))))
;; at this point have 1 running, 1 killed topo
(let [hist-topo-ids (vec (sort (.get_topo_ids (.getTopologyHistory (:nimbus cluster) (System/getProperty "user.name")))))]
(log-message "Checking user " (System/getProperty "user.name") " " hist-topo-ids)
@@ -515,22 +517,22 @@
(submit-local-topology (:nimbus cluster) "test" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 20} topology)
(bind storm-id (get-storm-id state "test"))
(advance-cluster-time cluster 15)
- (is (not-nil? (.storm-base state storm-id nil)))
- (is (not-nil? (.assignment-info state storm-id nil)))
+ (is (not-nil? (clojurify-storm-base (.stormBase state storm-id nil))))
+ (is (not-nil? (clojurify-assignment (.assignmentInfo state storm-id nil))))
(.killTopology (:nimbus cluster) "test")
;; check that storm is deactivated but alive
- (is (= :killed (-> (.storm-base state storm-id nil) :status :type)))
- (is (not-nil? (.assignment-info state storm-id nil)))
+ (is (= :killed (-> (clojurify-storm-base (.stormBase state storm-id nil)) :status :type)))
+ (is (not-nil? (clojurify-assignment (.assignmentInfo state storm-id nil))))
(advance-cluster-time cluster 18)
;; check that storm is deactivated but alive
- (is (= 1 (count (.heartbeat-storms state))))
+ (is (= 1 (count (.heartbeatStorms state))))
(advance-cluster-time cluster 3)
- (is (nil? (.storm-base state storm-id nil)))
- (is (nil? (.assignment-info state storm-id nil)))
+ (is (nil? (clojurify-storm-base (.stormBase state storm-id nil))))
+ (is (nil? (clojurify-assignment (.assignmentInfo state storm-id nil))))
;; cleanup happens on monitoring thread
(advance-cluster-time cluster 11)
- (is (empty? (.heartbeat-storms state)))
+ (is (empty? (.heartbeatStorms state)))
;; TODO: check that code on nimbus was cleaned up locally...
(is (thrown? NotAliveException (.killTopology (:nimbus cluster) "lalala")))
@@ -539,27 +541,27 @@
(is (thrown? AlreadyAliveException (submit-local-topology (:nimbus cluster) "2test" {} topology)))
(advance-cluster-time cluster 11)
(bind storm-id (get-storm-id state "2test"))
- (is (not-nil? (.storm-base state storm-id nil)))
+ (is (not-nil? (clojurify-storm-base (.stormBase state storm-id nil))))
(.killTopology (:nimbus cluster) "2test")
(is (thrown? AlreadyAliveException (submit-local-topology (:nimbus cluster) "2test" {} topology)))
(advance-cluster-time cluster 11)
- (is (= 1 (count (.heartbeat-storms state))))
+ (is (= 1 (count (.heartbeatStorms state))))
(advance-cluster-time cluster 6)
- (is (nil? (.storm-base state storm-id nil)))
- (is (nil? (.assignment-info state storm-id nil)))
+ (is (nil? (clojurify-storm-base (.stormBase state storm-id nil))))
+ (is (nil? (clojurify-assignment (.assignmentInfo state storm-id nil))))
(advance-cluster-time cluster 11)
- (is (= 0 (count (.heartbeat-storms state))))
+ (is (= 0 (count (.heartbeatStorms state))))
(submit-local-topology (:nimbus cluster) "test3" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 5} topology)
(bind storm-id3 (get-storm-id state "test3"))
(advance-cluster-time cluster 11)
- (.remove-storm! state storm-id3)
- (is (nil? (.storm-base state storm-id3 nil)))
- (is (nil? (.assignment-info state storm-id3 nil)))
+ (.removeStorm state storm-id3)
+ (is (nil? (clojurify-storm-base (.stormBase state storm-id3 nil))))
+ (is (nil? (clojurify-assignment (.assignmentInfo state storm-id3 nil))))
(advance-cluster-time cluster 11)
- (is (= 0 (count (.heartbeat-storms state))))
+ (is (= 0 (count (.heartbeatStorms state))))
;; this guarantees that monitor thread won't trigger for 10 more seconds
(advance-time-secs! 11)
@@ -575,9 +577,9 @@
(.killTopology (:nimbus cluster) "test3")
(advance-cluster-time cluster 6)
- (is (= 1 (count (.heartbeat-storms state))))
+ (is (= 1 (count (.heartbeatStorms state))))
(advance-cluster-time cluster 5)
- (is (= 0 (count (.heartbeat-storms state))))
+ (is (= 0 (count (.heartbeatStorms state))))
;; test kill with opts
(submit-local-topology (:nimbus cluster) "test4" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 100} topology)
@@ -585,9 +587,9 @@
(.killTopologyWithOpts (:nimbus cluster) "test4" (doto (KillOptions.) (.set_wait_secs 10)))
(bind storm-id4 (get-storm-id state "test4"))
(advance-cluster-time cluster 9)
- (is (not-nil? (.assignment-info state storm-id4 nil)))
+ (is (not-nil? (clojurify-assignment (.assignmentInfo state storm-id4 nil))))
(advance-cluster-time cluster 2)
- (is (nil? (.assignment-info state storm-id4 nil)))
+ (is (nil? (clojurify-assignment (.assignmentInfo state storm-id4 nil))))
)))
(deftest test-reassignment
@@ -906,7 +908,7 @@
(let [assignments (.assignments state nil)]
(log-message "Assignemts: " assignments)
(let [id->node->ports (into {} (for [id assignments
- :let [executor->node+port (:executor->node+port (.assignment-info state id nil))
+ :let [executor->node+port (:executor->node+port (clojurify-assignment (.assignmentInfo state id nil)))
node+ports (set (.values executor->node+port))
node->ports (apply merge-with (fn [a b] (distinct (concat a b))) (for [[node port] node+ports] {node [port]}))]]
{id node->ports}))
@@ -1029,7 +1031,7 @@
STORM-CLUSTER-MODE "local"
STORM-ZOOKEEPER-PORT zk-port
STORM-LOCAL-DIR nimbus-dir}))
- (bind cluster-state (cluster/mk-storm-cluster-state conf))
+ (bind cluster-state (StormZkClusterState. conf nil (ClusterStateContext.)))
(bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
(bind topology (thrift/mk-topology
{"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
@@ -1043,7 +1045,7 @@
(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))))
+ (is ( = #{storm-id2} (set (.activeStorms cluster-state))))
(.shutdown nimbus)
(.disconnect cluster-state)
)))))
@@ -1101,7 +1103,7 @@
STORM-CLUSTER-MODE "local"
STORM-ZOOKEEPER-PORT zk-port
STORM-LOCAL-DIR nimbus-dir}))
- (bind cluster-state (cluster/mk-storm-cluster-state conf))
+ (bind cluster-state (StormZkClusterState. conf nil (ClusterStateContext.)))
(bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
(bind topology (thrift/mk-topology
{"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
@@ -1111,7 +1113,7 @@
(zkLeaderElectorImpl [conf] (mock-leader-elector :is-leader false))))]
(letlocals
- (bind non-leader-cluster-state (cluster/mk-storm-cluster-state conf))
+ (bind non-leader-cluster-state (StormZkClusterState. conf nil (ClusterStateContext.)))
(bind non-leader-nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
;first we verify that the master nimbus can perform all actions, even with another nimbus present.
@@ -1347,13 +1349,15 @@
STORM-PRINCIPAL-TO-LOCAL-PLUGIN "org.apache.storm.security.auth.DefaultPrincipalToLocal"
NIMBUS-THRIFT-PORT 6666})
expected-acls nimbus/NIMBUS-ZK-ACLS
- fake-inimbus (reify INimbus (getForcedScheduler [this] nil))]
+ fake-inimbus (reify INimbus (getForcedScheduler [this] nil))
+ storm-zk (Mockito/mock Cluster)]
(with-open [_ (proxy [MockedConfigUtils] []
(nimbusTopoHistoryStateImpl [conf] nil))
zk-le (MockedZookeeper. (proxy [Zookeeper] []
- (zkLeaderElectorImpl [conf] nil)))]
+ (zkLeaderElectorImpl [conf] nil)))
+ storm-zk-le (MockedCluster. storm-zk)]
(stubbing [mk-authorization-handler nil
- cluster/mk-storm-cluster-state nil
+ ; cluster/mk-storm-cluster-state nil
nimbus/file-cache-map nil
nimbus/mk-blob-cache-map nil
nimbus/mk-bloblist-cache-map nil
@@ -1362,9 +1366,11 @@
mk-timer nil
nimbus/mk-scheduler nil]
(nimbus/nimbus-data auth-conf fake-inimbus)
- (verify-call-times-for cluster/mk-storm-cluster-state 1)
- (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
- expected-acls))))))
+ (.mkStormClusterStateImpl (Mockito/verify storm-zk (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
+ ; (verify-call-times-for cluster/mk-storm-cluster-state 1)
+ ; (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
+ ; expected-acls)
+ )))))
(deftest test-file-bogus-download
(with-local-cluster [cluster :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0 TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
@@ -1395,7 +1401,7 @@
STORM-CLUSTER-MODE "local"
STORM-ZOOKEEPER-PORT zk-port
STORM-LOCAL-DIR nimbus-dir}))
- (bind cluster-state (cluster/mk-storm-cluster-state conf))
+ (bind cluster-state (StormZkClusterState. conf nil (ClusterStateContext.)))
(bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
(sleep-secs 1)
(bind topology (thrift/mk-topology
@@ -1427,7 +1433,7 @@
STORM-ZOOKEEPER-PORT zk-port
STORM-LOCAL-DIR nimbus-dir
NIMBUS-TOPOLOGY-ACTION-NOTIFIER-PLUGIN (.getName InMemoryTopologyActionNotifier)}))
- (bind cluster-state (cluster/mk-storm-cluster-state conf))
+ (bind cluster-state (StormZkClusterState. conf nil (ClusterStateContext.)))
(bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
(bind notifier (InMemoryTopologyActionNotifier.))
(sleep-secs 1)
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/test/clj/org/apache/storm/security/auth/nimbus_auth_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/security/auth/nimbus_auth_test.clj b/storm-core/test/clj/org/apache/storm/security/auth/nimbus_auth_test.clj
index 361c4be..307296a 100644
--- a/storm-core/test/clj/org/apache/storm/security/auth/nimbus_auth_test.clj
+++ b/storm-core/test/clj/org/apache/storm/security/auth/nimbus_auth_test.clj
@@ -17,7 +17,6 @@
(:use [clojure test])
(:require [org.apache.storm [testing :as testing]])
(:require [org.apache.storm.daemon [nimbus :as nimbus]])
- (:require [org.apache.storm [zookeeper :as zk]])
(:require [org.apache.storm.security.auth [auth-test :refer [nimbus-timeout]]])
(:import [java.nio ByteBuffer])
(:import [org.apache.storm Config])
@@ -25,7 +24,7 @@
(:import [org.apache.storm.generated NotAliveException])
(:import [org.apache.storm.security.auth AuthUtils ThriftServer ThriftClient
ReqContext ThriftConnectionType])
- (:use [org.apache.storm cluster util config log])
+ (:use [org.apache.storm util config log])
(:use [org.apache.storm.daemon common nimbus])
(:import [org.apache.storm.generated Nimbus Nimbus$Client Nimbus$Processor
AuthorizationException SubmitOptions TopologyInitialStatus KillOptions])
[21/27] storm git commit: Merge branch 'master' into ClusterUtils
Posted by bo...@apache.org.
Merge branch 'master' into ClusterUtils
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/3717f393
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/3717f393
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/3717f393
Branch: refs/heads/master
Commit: 3717f3931e51825f16739cbed138e9789b7d1428
Parents: b7bc9bf 4ca7522
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Sun Feb 21 12:59:50 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Sun Feb 21 12:59:50 2016 +0800
----------------------------------------------------------------------
CHANGELOG.md | 3 +
external/storm-hdfs/pom.xml | 23 ++---
.../clj/org/apache/storm/daemon/supervisor.clj | 48 +++++++---
storm-core/src/clj/org/apache/storm/event.clj | 71 --------------
storm-core/src/clj/org/apache/storm/ui/core.clj | 6 +-
.../org/apache/storm/event/EventManager.java | 24 +++++
.../org/apache/storm/event/EventManagerImp.java | 97 ++++++++++++++++++++
7 files changed, 173 insertions(+), 99 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/3717f393/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
index f429d09,21e5854..a6f3a62
--- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
@@@ -34,7 -35,7 +35,7 @@@
(:use [org.apache.storm.daemon common])
(:import [org.apache.storm.command HealthCheck])
(:require [org.apache.storm.daemon [worker :as worker]]
- [org.apache.storm [process-simulator :as psim] [event :as event]]
- [org.apache.storm [process-simulator :as psim] [cluster :as cluster]]
++ [org.apache.storm [process-simulator :as psim]]
[clojure.set :as set])
(:import [org.apache.thrift.transport TTransportException])
(:import [org.apache.zookeeper data.ACL ZooDefs$Ids ZooDefs$Perms])
http://git-wip-us.apache.org/repos/asf/storm/blob/3717f393/storm-core/src/clj/org/apache/storm/ui/core.clj
----------------------------------------------------------------------
[05/27] storm git commit: callback maybe null
Posted by bo...@apache.org.
callback maybe null
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/5916b0b8
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/5916b0b8
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/5916b0b8
Branch: refs/heads/master
Commit: 5916b0b8089f9dd184fdd1ab2f18eb5e5deabc65
Parents: 9a79fb7
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Wed Feb 3 21:15:41 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Wed Feb 3 21:15:41 2016 +0800
----------------------------------------------------------------------
.../src/jvm/org/apache/storm/cluster/StormZkClusterState.java | 6 ++++--
1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/5916b0b8/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
index 3f32fe1..3a4205b 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
@@ -148,12 +148,14 @@ public class StormZkClusterState implements StormClusterState {
protected void issueCallback(AtomicReference<IFn> cb) {
IFn callback = cb.getAndSet(null);
- callback.invoke();
+ if (callback != null)
+ callback.invoke();
}
protected void issueMapCallback(ConcurrentHashMap<String, IFn> callbackConcurrentHashMap, String key) {
IFn callback = callbackConcurrentHashMap.remove(key);
- callback.invoke();
+ if (callback != null)
+ callback.invoke();
}
@Override
[25/27] storm git commit: Merge branch 'master' into ClusterUtils
Posted by bo...@apache.org.
Merge branch 'master' into ClusterUtils
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/0af67994
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/0af67994
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/0af67994
Branch: refs/heads/master
Commit: 0af6799411c747a6c900438ded222b7d757c7961
Parents: b09e275 11232b5
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Wed Feb 24 11:20:00 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Wed Feb 24 11:20:00 2016 +0800
----------------------------------------------------------------------
CHANGELOG.md | 4 +-
README.markdown | 3 +-
.../org/apache/storm/kafka/IntSerializer.java | 10 +-
.../src/jvm/org/apache/storm/utils/Time.java | 1 +
.../src/jvm/org/apache/storm/utils/Utils.java | 17 +-
.../test/clj/org/apache/storm/utils_test.clj | 111 ----------
.../jvm/org/apache/storm/utils/TimeTest.java | 112 ++++++++++
.../jvm/org/apache/storm/utils/UtilsTest.java | 219 +++++++++++++++++++
8 files changed, 353 insertions(+), 124 deletions(-)
----------------------------------------------------------------------
[20/27] storm git commit: resolve my little fault
Posted by bo...@apache.org.
resolve my little fault
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/b7bc9bf9
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/b7bc9bf9
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/b7bc9bf9
Branch: refs/heads/master
Commit: b7bc9bf9bd0793f2acbf158651e3200b30f97d99
Parents: cc1f6d7
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Sun Feb 21 10:30:05 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Sun Feb 21 10:30:05 2016 +0800
----------------------------------------------------------------------
storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/b7bc9bf9/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java b/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
index aae4231..1095fff 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
@@ -94,7 +94,7 @@ public class ClusterUtils {
public static List<ACL> mkTopoOnlyAcls(Map topoConf) throws NoSuchAlgorithmException {
List<ACL> aclList = null;
String payload = (String) topoConf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD);
- if (Utils.isZkAuthenticationConfiguredStormServer(topoConf)) {
+ if (Utils.isZkAuthenticationConfiguredTopology(topoConf)) {
aclList = new ArrayList<>();
ACL acl1 = ZooDefs.Ids.CREATOR_ALL_ACL.get(0);
aclList.add(acl1);
[17/27] storm git commit: Merge branch 'master' into ClusterUtils
Posted by bo...@apache.org.
Merge branch 'master' into ClusterUtils
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/d5463879
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/d5463879
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/d5463879
Branch: refs/heads/master
Commit: d5463879c0441a5a57eb23653a70bcf95d2efbaf
Parents: e0f3cb5 4699990
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Wed Feb 17 11:28:10 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Wed Feb 17 11:28:10 2016 +0800
----------------------------------------------------------------------
CHANGELOG.md | 8 +
bin/storm-config.cmd | 4 +
bin/storm.cmd | 22 +-
bin/storm.py | 8 +-
.../spout/RandomNumberGeneratorSpout.java | 95 +++++
.../trident/TridentMinMaxOfDevicesTopology.java | 201 +++++++++++
.../TridentMinMaxOfVehiclesTopology.java | 180 ++++++++++
pom.xml | 6 +
storm-core/pom.xml | 9 +
.../clj/org/apache/storm/command/activate.clj | 24 --
.../clj/org/apache/storm/command/deactivate.clj | 24 --
.../org/apache/storm/command/kill_topology.clj | 29 --
.../src/clj/org/apache/storm/command/list.clj | 38 --
.../clj/org/apache/storm/daemon/executor.clj | 68 ++--
.../src/clj/org/apache/storm/daemon/worker.clj | 54 +--
.../src/clj/org/apache/storm/disruptor.clj | 89 -----
.../jvm/org/apache/storm/command/Activate.java | 40 +++
.../src/jvm/org/apache/storm/command/CLI.java | 353 +++++++++++++++++++
.../org/apache/storm/command/Deactivate.java | 40 +++
.../org/apache/storm/command/KillTopology.java | 51 +++
.../src/jvm/org/apache/storm/command/List.java | 50 +++
.../jvm/org/apache/storm/trident/Stream.java | 121 ++++++-
.../operation/builtin/ComparisonAggregator.java | 91 +++++
.../storm/trident/operation/builtin/Max.java | 37 ++
.../operation/builtin/MaxWithComparator.java | 51 +++
.../storm/trident/operation/builtin/Min.java | 36 ++
.../operation/builtin/MinWithComparator.java | 51 +++
.../org/apache/storm/utils/DisruptorQueue.java | 15 +-
.../org/apache/storm/utils/NimbusClient.java | 19 +-
.../src/jvm/org/apache/storm/utils/Utils.java | 20 +-
.../jvm/org/apache/storm/command/TestCLI.java | 59 ++++
31 files changed, 1593 insertions(+), 300 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/d5463879/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/executor.clj
index 33b89ed,3af365b..902650c
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@@ -38,8 -38,10 +38,9 @@@
(:import [org.apache.storm.grouping LoadAwareCustomStreamGrouping LoadAwareShuffleGrouping LoadMapping ShuffleGrouping])
(:import [java.lang Thread Thread$UncaughtExceptionHandler]
[java.util.concurrent ConcurrentLinkedQueue]
- [org.json.simple JSONValue])
- (:require [org.apache.storm [thrift :as thrift] [disruptor :as disruptor] [stats :as stats]])
+ [org.json.simple JSONValue]
+ [com.lmax.disruptor.dsl ProducerType])
- (:require [org.apache.storm [thrift :as thrift]
- [cluster :as cluster] [stats :as stats]])
++ (:require [org.apache.storm [thrift :as thrift] [stats :as stats]])
(:require [org.apache.storm.daemon [task :as task]])
(:require [org.apache.storm.daemon.builtin-metrics :as builtin-metrics])
(:require [clojure.set :as set]))
http://git-wip-us.apache.org/repos/asf/storm/blob/d5463879/storm-core/src/clj/org/apache/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/worker.clj
index 9863427,83ae9be..b80cd9e
--- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
@@@ -19,7 -19,7 +19,7 @@@
(:require [clj-time.core :as time])
(:require [clj-time.coerce :as coerce])
(:require [org.apache.storm.daemon [executor :as executor]])
- (:require [org.apache.storm [disruptor :as disruptor]])
- (:require [org.apache.storm [cluster :as cluster]])
++
(:require [clojure.set :as set])
(:import [java.util.concurrent Executors]
[org.apache.storm.hooks IWorkerHook BaseWorkerHook]
@@@ -244,13 -244,14 +244,14 @@@
)
:timer-name timer-name))
-(defn worker-data [conf mq-context storm-id assignment-id port worker-id storm-conf cluster-state storm-cluster-state]
+(defn worker-data [conf mq-context storm-id assignment-id port worker-id storm-conf state-store storm-cluster-state]
(let [assignment-versions (atom {})
executors (set (read-worker-executors storm-conf storm-cluster-state storm-id assignment-id port assignment-versions))
- transfer-queue (disruptor/disruptor-queue "worker-transfer-queue" (storm-conf TOPOLOGY-TRANSFER-BUFFER-SIZE)
+ transfer-queue (DisruptorQueue. "worker-transfer-queue"
+ (storm-conf TOPOLOGY-TRANSFER-BUFFER-SIZE)
(storm-conf TOPOLOGY-DISRUPTOR-WAIT-TIMEOUT-MILLIS)
- :batch-size (storm-conf TOPOLOGY-DISRUPTOR-BATCH-SIZE)
- :batch-timeout (storm-conf TOPOLOGY-DISRUPTOR-BATCH-TIMEOUT-MILLIS))
+ (storm-conf TOPOLOGY-DISRUPTOR-BATCH-SIZE)
+ (storm-conf TOPOLOGY-DISRUPTOR-BATCH-TIMEOUT-MILLIS))
executor-receive-queue-map (mk-receive-queue-map storm-conf executors)
receive-queue-map (->> executor-receive-queue-map
[11/27] storm git commit: Merge branch 'master' into ClusterUtils
Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/test/clj/org/apache/storm/supervisor_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/supervisor_test.clj
index b89b7bb,9c31ddf..3ebdbcd
--- a/storm-core/test/clj/org/apache/storm/supervisor_test.clj
+++ b/storm-core/test/clj/org/apache/storm/supervisor_test.clj
@@@ -21,20 -21,20 +21,23 @@@
(:require [clojure [string :as string] [set :as set]])
(:import [org.apache.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter TestPlannerSpout])
(:import [org.apache.storm.scheduler ISupervisor])
- (:import [org.apache.storm.utils ConfigUtils])
+ (:import [org.apache.storm.utils Time Utils$UptimeComputer ConfigUtils])
(:import [org.apache.storm.generated RebalanceOptions])
- (:import [org.apache.storm.testing.staticmocking MockedConfigUtils MockedCluster])
- (:import [org.mockito Matchers Mockito])
++ (:import [org.apache.storm.testing.staticmocking MockedCluster])
(:import [java.util UUID])
- (:import [org.mockito Mockito])
++ (:import [org.mockito Mockito Matchers])
+ (:import [org.mockito.exceptions.base MockitoAssertionError])
(:import [java.io File])
(:import [java.nio.file Files])
- (:import [org.apache.storm.cluster StormClusterStateImpl ClusterStateContext ClusterUtils])
- (:import [org.apache.storm.utils Utils IPredicate]
++ (:import [org.apache.storm.utils Utils IPredicate])
++ (:import [org.apache.storm.cluster StormClusterStateImpl ClusterStateContext ClusterUtils]
+ [org.apache.storm.utils.staticmocking ConfigUtilsInstaller
+ UtilsInstaller])
(:import [java.nio.file.attribute FileAttribute])
- (:use [org.apache.storm config testing util timer log])
+ (:use [org.apache.storm config testing util timer log converter])
(:use [org.apache.storm.daemon common])
(:require [org.apache.storm.daemon [worker :as worker] [supervisor :as supervisor]]
- [org.apache.storm [thrift :as thrift] [cluster :as cluster]])
+ [org.apache.storm [thrift :as thrift]])
(:use [conjure core])
(:require [clojure.java.io :as io]))
@@@ -43,9 -43,10 +46,10 @@@
[cluster supervisor-id port]
(let [state (:storm-cluster-state cluster)
slot-assigns (for [storm-id (.assignments state nil)]
- (let [executors (-> (.assignment-info state storm-id nil)
+ (let [executors (-> (clojurify-assignment (.assignmentInfo state storm-id nil))
:executor->node+port
- reverse-map
+ (Utils/reverseMap)
+ clojurify-structure
(get [supervisor-id port] ))]
(when executors [storm-id executors])
))
@@@ -565,198 -632,203 +635,201 @@@
fake-isupervisor (reify ISupervisor
(getSupervisorId [this] nil)
(getAssignmentId [this] nil))
+ fake-cu (proxy [ConfigUtils] []
+ (supervisorStateImpl [conf] nil)
+ (supervisorLocalDirImpl [conf] nil))
+ fake-utils (proxy [Utils] []
+ (localHostnameImpl [] nil)
+ (makeUptimeComputer [] (proxy [Utils$UptimeComputer] []
- (upTime [] 0))))]
++ (upTime [] 0))))
+ cluster-utils (Mockito/mock ClusterUtils)]
- (with-open [_ (proxy [MockedConfigUtils] []
- (supervisorStateImpl [conf] nil)
- (supervisorLocalDirImpl [conf] nil))
+ (with-open [_ (ConfigUtilsInstaller. fake-cu)
- _ (UtilsInstaller. fake-utils)]
- (stubbing [cluster/mk-storm-cluster-state nil
- mk-timer nil]
++ _ (UtilsInstaller. fake-utils)
+ mocked-cluster (MockedCluster. cluster-utils)]
- (stubbing [uptime-computer nil
- ; cluster/mk-storm-cluster-state nil
- local-hostname nil
- mk-timer nil]
(supervisor/supervisor-data auth-conf nil fake-isupervisor)
- (.mkStormClusterStateImpl (Mockito/verify cluster-utils (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
- ; (verify-call-times-for cluster/mk-storm-cluster-state 1)
- ; (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
- ; expected-acls)
- )))))
- (verify-call-times-for cluster/mk-storm-cluster-state 1)
- (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
- expected-acls)))))
++ (.mkStormClusterStateImpl (Mockito/verify cluster-utils (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))))))
- (deftest test-write-log-metadata
- (testing "supervisor writes correct data to logs metadata file"
- (let [exp-owner "alice"
- exp-worker-id "42"
- exp-storm-id "0123456789"
- exp-port 4242
- exp-logs-users ["bob" "charlie" "daryl"]
- exp-logs-groups ["read-only-group" "special-group"]
- storm-conf {TOPOLOGY-SUBMITTER-USER "alice"
- TOPOLOGY-USERS ["charlie" "bob"]
- TOPOLOGY-GROUPS ["special-group"]
- LOGS-GROUPS ["read-only-group"]
- LOGS-USERS ["daryl"]}
- exp-data {TOPOLOGY-SUBMITTER-USER exp-owner
- "worker-id" exp-worker-id
- LOGS-USERS exp-logs-users
- LOGS-GROUPS exp-logs-groups}
- conf {}]
- (mocking [supervisor/write-log-metadata-to-yaml-file!]
- (supervisor/write-log-metadata! storm-conf exp-owner exp-worker-id
- exp-storm-id exp-port conf)
- (verify-called-once-with-args supervisor/write-log-metadata-to-yaml-file!
- exp-storm-id exp-port exp-data conf)))))
+ (deftest test-write-log-metadata
+ (testing "supervisor writes correct data to logs metadata file"
+ (let [exp-owner "alice"
+ exp-worker-id "42"
+ exp-storm-id "0123456789"
+ exp-port 4242
+ exp-logs-users ["bob" "charlie" "daryl"]
+ exp-logs-groups ["read-only-group" "special-group"]
+ storm-conf {TOPOLOGY-SUBMITTER-USER "alice"
+ TOPOLOGY-USERS ["charlie" "bob"]
+ TOPOLOGY-GROUPS ["special-group"]
+ LOGS-GROUPS ["read-only-group"]
+ LOGS-USERS ["daryl"]}
+ exp-data {TOPOLOGY-SUBMITTER-USER exp-owner
+ "worker-id" exp-worker-id
+ LOGS-USERS exp-logs-users
+ LOGS-GROUPS exp-logs-groups}
+ conf {}]
+ (mocking [supervisor/write-log-metadata-to-yaml-file!]
+ (supervisor/write-log-metadata! storm-conf exp-owner exp-worker-id
+ exp-storm-id exp-port conf)
+ (verify-called-once-with-args supervisor/write-log-metadata-to-yaml-file!
+ exp-storm-id exp-port exp-data conf)))))
- (deftest test-worker-launcher-requires-user
- (testing "worker-launcher throws on blank user"
- (mocking [launch-process]
- (is (thrown-cause-with-msg? java.lang.IllegalArgumentException
- #"(?i).*user cannot be blank.*"
- (supervisor/worker-launcher {} nil ""))))))
+ (deftest test-worker-launcher-requires-user
+ (testing "worker-launcher throws on blank user"
+ (let [utils-proxy (proxy [Utils] []
+ (launchProcessImpl [& _] nil))]
+ (with-open [_ (UtilsInstaller. utils-proxy)]
+ (is (try
+ (supervisor/worker-launcher {} nil "")
+ false
+ (catch Throwable t
+ (and (re-matches #"(?i).*user cannot be blank.*" (.getMessage t))
+ (Utils/exceptionCauseIsInstanceOf java.lang.IllegalArgumentException t)))))))))
- (defn found? [sub-str input-str]
- (if (string? input-str)
- (contrib-str/substring? sub-str (str input-str))
- (boolean (some #(contrib-str/substring? sub-str %) input-str))))
+ (defn found? [sub-str input-str]
+ (if (string? input-str)
+ (contrib-str/substring? sub-str (str input-str))
+ (boolean (some #(contrib-str/substring? sub-str %) input-str))))
- (defn not-found? [sub-str input-str]
+ (defn not-found? [sub-str input-str]
(complement (found? sub-str input-str)))
- (deftest test-substitute-childopts-happy-path-string
- (testing "worker-launcher replaces ids in childopts"
- (let [worker-id "w-01"
- topology-id "s-01"
- port 9999
- mem-onheap 512
- childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log -Xms256m -Xmx%HEAP-MEM%m"
- expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
- childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
- (is (= expected-childopts childopts-with-ids)))))
+ (deftest test-substitute-childopts-happy-path-string
+ (testing "worker-launcher replaces ids in childopts"
+ (let [worker-id "w-01"
+ topology-id "s-01"
+ port 9999
+ mem-onheap 512
+ childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log -Xms256m -Xmx%HEAP-MEM%m"
+ expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
+ childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+ (is (= expected-childopts childopts-with-ids)))))
- (deftest test-substitute-childopts-happy-path-list
- (testing "worker-launcher replaces ids in childopts"
- (let [worker-id "w-01"
- topology-id "s-01"
- port 9999
- mem-onheap 512
- childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m" "-Xmx%HEAP-MEM%m")
- expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
- childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
- (is (= expected-childopts childopts-with-ids)))))
+ (deftest test-substitute-childopts-happy-path-list
+ (testing "worker-launcher replaces ids in childopts"
+ (let [worker-id "w-01"
+ topology-id "s-01"
+ port 9999
+ mem-onheap 512
+ childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m" "-Xmx%HEAP-MEM%m")
+ expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
+ childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+ (is (= expected-childopts childopts-with-ids)))))
- (deftest test-substitute-childopts-happy-path-list-arraylist
- (testing "worker-launcher replaces ids in childopts"
- (let [worker-id "w-01"
- topology-id "s-01"
- port 9999
- mem-onheap 512
- childopts '["-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m" "-Xmx%HEAP-MEM%m"]
- expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
- childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
- (is (= expected-childopts childopts-with-ids)))))
+ (deftest test-substitute-childopts-happy-path-list-arraylist
+ (testing "worker-launcher replaces ids in childopts"
+ (let [worker-id "w-01"
+ topology-id "s-01"
+ port 9999
+ mem-onheap 512
+ childopts '["-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m" "-Xmx%HEAP-MEM%m"]
+ expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
+ childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+ (is (= expected-childopts childopts-with-ids)))))
- (deftest test-substitute-childopts-topology-id-alone
- (testing "worker-launcher replaces ids in childopts"
- (let [worker-id "w-01"
- topology-id "s-01"
- port 9999
- mem-onheap 512
- childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%TOPOLOGY-ID%.log"
- expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-s-01.log")
- childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
- (is (= expected-childopts childopts-with-ids)))))
+ (deftest test-substitute-childopts-topology-id-alone
+ (testing "worker-launcher replaces ids in childopts"
+ (let [worker-id "w-01"
+ topology-id "s-01"
+ port 9999
+ mem-onheap 512
+ childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%TOPOLOGY-ID%.log"
+ expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-s-01.log")
+ childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+ (is (= expected-childopts childopts-with-ids)))))
- (deftest test-substitute-childopts-no-keys
- (testing "worker-launcher has no ids to replace in childopts"
- (let [worker-id "w-01"
- topology-id "s-01"
- port 9999
- mem-onheap 512
- childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log"
- expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log")
- childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
- (is (= expected-childopts childopts-with-ids)))))
+ (deftest test-substitute-childopts-no-keys
+ (testing "worker-launcher has no ids to replace in childopts"
+ (let [worker-id "w-01"
+ topology-id "s-01"
+ port 9999
+ mem-onheap 512
+ childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log"
+ expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log")
+ childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+ (is (= expected-childopts childopts-with-ids)))))
- (deftest test-substitute-childopts-nil-childopts
- (testing "worker-launcher has nil childopts"
- (let [worker-id "w-01"
- topology-id "s-01"
- port 9999
- mem-onheap 512
- childopts nil
- expected-childopts nil
- childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
- (is (= expected-childopts childopts-with-ids)))))
+ (deftest test-substitute-childopts-nil-childopts
+ (testing "worker-launcher has nil childopts"
+ (let [worker-id "w-01"
+ topology-id "s-01"
+ port 9999
+ mem-onheap 512
+ childopts nil
+ expected-childopts nil
+ childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+ (is (= expected-childopts childopts-with-ids)))))
- (deftest test-substitute-childopts-nil-ids
- (testing "worker-launcher has nil ids"
- (let [worker-id nil
- topology-id "s-01"
- port 9999
- mem-onheap 512
- childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log"
- expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01--9999.log")
- childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
- (is (= expected-childopts childopts-with-ids)))))
+ (deftest test-substitute-childopts-nil-ids
+ (testing "worker-launcher has nil ids"
+ (let [worker-id nil
+ topology-id "s-01"
+ port 9999
+ mem-onheap 512
+ childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log"
+ expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01--9999.log")
+ childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+ (is (= expected-childopts childopts-with-ids)))))
- (deftest test-retry-read-assignments
- (with-simulated-time-local-cluster [cluster
- :supervisors 0
- :ports-per-supervisor 2
- :daemon-conf {ConfigUtils/NIMBUS_DO_NOT_REASSIGN true
- NIMBUS-MONITOR-FREQ-SECS 10
- TOPOLOGY-MESSAGE-TIMEOUT-SECS 30
- TOPOLOGY-ACKER-EXECUTORS 0}]
- (letlocals
- (bind sup1 (add-supervisor cluster :id "sup1" :ports [1 2 3 4]))
- (bind topology1 (thrift/mk-topology
- {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)}
- {}))
- (bind topology2 (thrift/mk-topology
- {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)}
- {}))
- (bind state (:storm-cluster-state cluster))
- (bind changed (capture-changed-workers
- (submit-mocked-assignment
- (:nimbus cluster)
- (:storm-cluster-state cluster)
- "topology1"
- {TOPOLOGY-WORKERS 2}
- topology1
- {1 "1"
- 2 "1"}
- {[1 1] ["sup1" 1]
- [2 2] ["sup1" 2]}
- {["sup1" 1] [0.0 0.0 0.0]
- ["sup1" 2] [0.0 0.0 0.0]
- })
- (submit-mocked-assignment
- (:nimbus cluster)
- (:storm-cluster-state cluster)
- "topology2"
- {TOPOLOGY-WORKERS 2}
- topology2
- {1 "1"
- 2 "1"}
- {[1 1] ["sup1" 1]
- [2 2] ["sup1" 2]}
- {["sup1" 1] [0.0 0.0 0.0]
- ["sup1" 2] [0.0 0.0 0.0]
- })
- ;; Instead of sleeping until topology is scheduled, rebalance topology so mk-assignments is called.
- (.rebalance (:nimbus cluster) "topology1" (doto (RebalanceOptions.) (.set_wait_secs 0)))
- ))
- (is (empty? (:launched changed)))
- (bind options (RebalanceOptions.))
- (.set_wait_secs options 0)
- (bind changed (capture-changed-workers
- (.rebalance (:nimbus cluster) "topology2" options)
- (advance-cluster-time cluster 10)
- (heartbeat-workers cluster "sup1" [1 2 3 4])
- (advance-cluster-time cluster 10)
- ))
- (validate-launched-once (:launched changed)
- {"sup1" [1 2]}
- (get-storm-id (:storm-cluster-state cluster) "topology1"))
- (validate-launched-once (:launched changed)
- {"sup1" [3 4]}
- (get-storm-id (:storm-cluster-state cluster) "topology2"))
- )))
+ (deftest test-retry-read-assignments
+ (with-simulated-time-local-cluster [cluster
+ :supervisors 0
+ :ports-per-supervisor 2
+ :daemon-conf {ConfigUtils/NIMBUS_DO_NOT_REASSIGN true
+ NIMBUS-MONITOR-FREQ-SECS 10
+ TOPOLOGY-MESSAGE-TIMEOUT-SECS 30
+ TOPOLOGY-ACKER-EXECUTORS 0}]
+ (letlocals
+ (bind sup1 (add-supervisor cluster :id "sup1" :ports [1 2 3 4]))
+ (bind topology1 (thrift/mk-topology
+ {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)}
+ {}))
+ (bind topology2 (thrift/mk-topology
+ {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)}
+ {}))
+ (bind state (:storm-cluster-state cluster))
+ (bind changed (capture-changed-workers
+ (submit-mocked-assignment
+ (:nimbus cluster)
+ (:storm-cluster-state cluster)
+ "topology1"
+ {TOPOLOGY-WORKERS 2}
+ topology1
+ {1 "1"
+ 2 "1"}
+ {[1 1] ["sup1" 1]
+ [2 2] ["sup1" 2]}
+ {["sup1" 1] [0.0 0.0 0.0]
+ ["sup1" 2] [0.0 0.0 0.0]
+ })
+ (submit-mocked-assignment
+ (:nimbus cluster)
+ (:storm-cluster-state cluster)
+ "topology2"
+ {TOPOLOGY-WORKERS 2}
+ topology2
+ {1 "1"
+ 2 "1"}
+ {[1 1] ["sup1" 1]
+ [2 2] ["sup1" 2]}
+ {["sup1" 1] [0.0 0.0 0.0]
+ ["sup1" 2] [0.0 0.0 0.0]
+ })
+ ;; Instead of sleeping until topology is scheduled, rebalance topology so mk-assignments is called.
+ (.rebalance (:nimbus cluster) "topology1" (doto (RebalanceOptions.) (.set_wait_secs 0)))
+ ))
+ (is (empty? (:launched changed)))
+ (bind options (RebalanceOptions.))
+ (.set_wait_secs options 0)
+ (bind changed (capture-changed-workers
+ (.rebalance (:nimbus cluster) "topology2" options)
+ (advance-cluster-time cluster 10)
+ (heartbeat-workers cluster "sup1" [1 2 3 4])
+ (advance-cluster-time cluster 10)
+ ))
+ (validate-launched-once (:launched changed)
+ {"sup1" [1 2]}
+ (get-storm-id (:storm-cluster-state cluster) "topology1"))
+ (validate-launched-once (:launched changed)
+ {"sup1" [3 4]}
+ (get-storm-id (:storm-cluster-state cluster) "topology2"))
- ))))
++ )))
[24/27] storm git commit: update/fix some codes based on @revans2
Posted by bo...@apache.org.
update/fix some codes based on @revans2
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/b09e2755
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/b09e2755
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/b09e2755
Branch: refs/heads/master
Commit: b09e2755c2fedcb9048afda63bea897fe1dbd12b
Parents: d709525
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Wed Feb 24 11:19:06 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Wed Feb 24 11:19:06 2016 +0800
----------------------------------------------------------------------
.../src/clj/org/apache/storm/daemon/nimbus.clj | 2 +-
.../clj/org/apache/storm/daemon/supervisor.clj | 4 +-
.../src/clj/org/apache/storm/daemon/worker.clj | 8 +-
.../org/apache/storm/cluster/ClusterUtils.java | 25 +-----
.../storm/cluster/IStormClusterState.java | 4 +-
.../storm/cluster/PaceMakerStateStorage.java | 12 ++-
.../storm/cluster/StormClusterStateImpl.java | 41 +++++-----
.../apache/storm/cluster/ZKStateStorage.java | 80 +++++++-------------
.../org/apache/storm/zookeeper/Zookeeper.java | 9 ++-
9 files changed, 74 insertions(+), 111 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/b09e2755/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index 2f6587a..e524ec2 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@ -1676,7 +1676,7 @@
[(node->host node) port])
executor->node+port)
nodeinfos (stats/extract-nodeinfos-from-hb-for-comp executor->host+port task->component false component_id)
- all-pending-actions-for-topology (clojurify-profile-request (.getTopologyProfileRequests storm-cluster-state id true))
+ all-pending-actions-for-topology (clojurify-profile-request (.getTopologyProfileRequests storm-cluster-state id))
latest-profile-actions (remove nil? (map (fn [nodeInfo]
(->> all-pending-actions-for-topology
(filter #(and (= (:host nodeInfo) (.get_node (.get_nodeInfo %)))
http://git-wip-us.apache.org/repos/asf/storm/blob/b09e2755/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
index 0cee414..1446ac9 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
@@ -81,7 +81,7 @@
(->>
(dofor [sid (distinct storm-ids)]
- (if-let [topo-profile-actions (into [] (for [request (.getTopologyProfileRequests storm-cluster-state sid false)] (clojurify-profile-request request)))]
+ (if-let [topo-profile-actions (into [] (for [request (.getTopologyProfileRequests storm-cluster-state sid)] (clojurify-profile-request request)))]
{sid topo-profile-actions}))
(apply merge))]
{:assignments (into {} (for [[k v] new-assignments] [k (:data v)]))
@@ -607,7 +607,7 @@
storm-cluster-state (:storm-cluster-state supervisor)
^ISupervisor isupervisor (:isupervisor supervisor)
^LocalState local-state (:local-state supervisor)
- sync-callback (fn [& ignored] (.add event-manager (reify Runnable
+ sync-callback (fn [] (.add event-manager (reify Runnable
(^void run [this]
(callback-supervisor)))))
assignment-versions @(:assignment-versions supervisor)
http://git-wip-us.apache.org/repos/asf/storm/blob/b09e2755/storm-core/src/clj/org/apache/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/worker.clj b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
index af88f6a..110d415 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
@@ -383,7 +383,7 @@
storm-id (:storm-id worker)]
(fn refresh-connections
([]
- (refresh-connections (fn [& ignored]
+ (refresh-connections (fn []
(.schedule
(:refresh-connections-timer worker) 0 refresh-connections))))
([callback]
@@ -438,7 +438,7 @@
(defn refresh-storm-active
([worker]
(refresh-storm-active
- worker (fn [& ignored]
+ worker (fn []
(.schedule
(:refresh-active-timer worker) 0 (partial refresh-storm-active worker)))))
([worker callback]
@@ -685,7 +685,7 @@
backpressure-thread (WorkerBackpressureThread. (:backpressure-trigger worker) worker backpressure-handler)
_ (if ((:storm-conf worker) TOPOLOGY-BACKPRESSURE-ENABLE)
(.start backpressure-thread))
- callback (fn cb [& ignored]
+ callback (fn cb []
(let [throttle-on (.topologyBackpressure storm-cluster-state storm-id cb)]
(reset! (:throttle-on worker) throttle-on)))
_ (if ((:storm-conf worker) TOPOLOGY-BACKPRESSURE-ENABLE)
@@ -757,7 +757,7 @@
(dofor [e @executors] (.credentials-changed e new-creds))
(reset! credentials new-creds))))
check-throttle-changed (fn []
- (let [callback (fn cb [& ignored]
+ (let [callback (fn cb []
(let [throttle-on (.topologyBackpressure (:storm-cluster-state worker) storm-id cb)]
(reset! (:throttle-on worker) throttle-on)))
new-throttle-on (.topologyBackpressure (:storm-cluster-state worker) storm-id callback)]
http://git-wip-us.apache.org/repos/asf/storm/blob/b09e2755/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java b/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
index 1095fff..96c177b 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
@@ -211,7 +211,6 @@ public class ClusterUtils {
IStateStorage Storage = _instance.mkStateStorageImpl((Map) stateStorage, (Map) stateStorage, acls, context);
return new StormClusterStateImpl(Storage, acls, context, true);
}
-
}
public IStateStorage mkStateStorageImpl(Map config, Map auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
@@ -237,25 +236,9 @@ public class ClusterUtils {
}
public static String stringifyError(Throwable error) {
- String errorString = null;
- StringWriter result = null;
- PrintWriter printWriter = null;
- try {
- result = new StringWriter();
- printWriter = new PrintWriter(result);
- error.printStackTrace(printWriter);
- if (result != null) {
- errorString = result.toString();
- }
- } finally {
- try {
- if (result != null)
- result.close();
- if (printWriter != null)
- printWriter.close();
- } catch (Exception e) {
- }
- }
- return errorString;
+ StringWriter result = new StringWriter();
+ PrintWriter printWriter = new PrintWriter(result);
+ error.printStackTrace(printWriter);
+ return result.toString();
}
}
http://git-wip-us.apache.org/repos/asf/storm/blob/b09e2755/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
index c88935e..e26c598 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
@@ -45,9 +45,9 @@ public interface IStormClusterState {
public ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, String node, Long port);
- public List<ProfileRequest> getWorkerProfileRequests(String stormId, NodeInfo nodeInfo, boolean isThrift);
+ public List<ProfileRequest> getWorkerProfileRequests(String stormId, NodeInfo nodeInfo);
- public List<ProfileRequest> getTopologyProfileRequests(String stormId, boolean isThrift);
+ public List<ProfileRequest> getTopologyProfileRequests(String stormId);
public void setWorkerProfileRequest(String stormId, ProfileRequest profileRequest);
http://git-wip-us.apache.org/repos/asf/storm/blob/b09e2755/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java b/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
index c29078e..c42bd38 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
@@ -127,7 +127,8 @@ public class PaceMakerStateStorage implements IStateStorage {
if (retry <= 0) {
throw Utils.wrapInRuntime(e);
}
- LOG.error("{} Failed to set_worker_hb. Will make {} more attempts.", e.getMessage(), retry--);
+ retry--;
+ LOG.error("{} Failed to set_worker_hb. Will make {} more attempts.", e.getMessage(), retry);
}
}
}
@@ -148,7 +149,8 @@ public class PaceMakerStateStorage implements IStateStorage {
if (retry <= 0) {
throw Utils.wrapInRuntime(e);
}
- LOG.error("{} Failed to get_worker_hb. Will make {} more attempts.", e.getMessage(), retry--);
+ retry--;
+ LOG.error("{} Failed to get_worker_hb. Will make {} more attempts.", e.getMessage(), retry);
}
}
}
@@ -169,7 +171,8 @@ public class PaceMakerStateStorage implements IStateStorage {
if (retry <= 0) {
throw Utils.wrapInRuntime(e);
}
- LOG.error("{} Failed to get_worker_hb_children. Will make {} more attempts.", e.getMessage(), retry--);
+ retry--;
+ LOG.error("{} Failed to get_worker_hb_children. Will make {} more attempts.", e.getMessage(), retry);
}
}
}
@@ -190,7 +193,8 @@ public class PaceMakerStateStorage implements IStateStorage {
if (retry <= 0) {
throw Utils.wrapInRuntime(e);
}
- LOG.error("{} Failed to delete_worker_hb. Will make {} more attempts.", e.getMessage(), retry--);
+ retry--;
+ LOG.error("{} Failed to delete_worker_hb. Will make {} more attempts.", e.getMessage(), retry);
}
}
}
http://git-wip-us.apache.org/repos/asf/storm/blob/b09e2755/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
index 5fa586a..bde7670 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
@@ -106,7 +106,7 @@ public class StormClusterStateImpl implements IStormClusterState {
} else if (root.equals(ClusterUtils.LOGCONFIG_ROOT) && size > 1) {
issueMapCallback(logConfigCallback, toks.get(1));
} else if (root.equals(ClusterUtils.BACKPRESSURE_ROOT) && size > 1) {
- issueMapCallback(logConfigCallback, toks.get(1));
+ issueMapCallback(backPressureCallback, toks.get(1));
} else {
LOG.error("{} Unknown callback for subtree {}", new RuntimeException("Unknown callback for this path"), path);
Runtime.getRuntime().exit(30);
@@ -242,9 +242,9 @@ public class StormClusterStateImpl implements IStormClusterState {
}
@Override
- public List<ProfileRequest> getWorkerProfileRequests(String stormId, NodeInfo nodeInfo, boolean isThrift) {
+ public List<ProfileRequest> getWorkerProfileRequests(String stormId, NodeInfo nodeInfo) {
List<ProfileRequest> requests = new ArrayList<>();
- List<ProfileRequest> profileRequests = getTopologyProfileRequests(stormId, isThrift);
+ List<ProfileRequest> profileRequests = getTopologyProfileRequests(stormId);
for (ProfileRequest profileRequest : profileRequests) {
NodeInfo nodeInfo1 = profileRequest.get_nodeInfo();
if (nodeInfo1.equals(nodeInfo))
@@ -254,7 +254,7 @@ public class StormClusterStateImpl implements IStormClusterState {
}
@Override
- public List<ProfileRequest> getTopologyProfileRequests(String stormId, boolean isThrift) {
+ public List<ProfileRequest> getTopologyProfileRequests(String stormId) {
List<ProfileRequest> profileRequests = new ArrayList<>();
String path = ClusterUtils.profilerConfigPath(stormId);
if (stateStorage.node_exists(path, false)) {
@@ -382,6 +382,9 @@ public class StormClusterStateImpl implements IStormClusterState {
@Override
public LogConfig topologyLogConfig(String stormId, Runnable cb) {
+ if (cb != null){
+ logConfigCallback.put(stormId, cb);
+ }
String path = ClusterUtils.logConfigPath(stormId);
return ClusterUtils.maybeDeserialize(stateStorage.get_data(path, cb != null), LogConfig.class);
}
@@ -625,25 +628,21 @@ public class StormClusterStateImpl implements IStormClusterState {
@Override
public List<ErrorInfo> errors(String stormId, String componentId) {
List<ErrorInfo> errorInfos = new ArrayList<>();
- try {
- String path = ClusterUtils.errorPath(stormId, componentId);
- if (stateStorage.node_exists(path, false)) {
- List<String> childrens = stateStorage.get_children(path, false);
- for (String child : childrens) {
- String childPath = path + ClusterUtils.ZK_SEPERATOR + child;
- ErrorInfo errorInfo = ClusterUtils.maybeDeserialize(stateStorage.get_data(childPath, false), ErrorInfo.class);
- if (errorInfo != null)
- errorInfos.add(errorInfo);
- }
+ String path = ClusterUtils.errorPath(stormId, componentId);
+ if (stateStorage.node_exists(path, false)) {
+ List<String> childrens = stateStorage.get_children(path, false);
+ for (String child : childrens) {
+ String childPath = path + ClusterUtils.ZK_SEPERATOR + child;
+ ErrorInfo errorInfo = ClusterUtils.maybeDeserialize(stateStorage.get_data(childPath, false), ErrorInfo.class);
+ if (errorInfo != null)
+ errorInfos.add(errorInfo);
}
- Collections.sort(errorInfos, new Comparator<ErrorInfo>() {
- public int compare(ErrorInfo arg0, ErrorInfo arg1) {
- return Integer.compare(arg1.get_error_time_secs(), arg0.get_error_time_secs());
- }
- });
- } catch (Exception e) {
- throw Utils.wrapInRuntime(e);
}
+ Collections.sort(errorInfos, new Comparator<ErrorInfo>() {
+ public int compare(ErrorInfo arg0, ErrorInfo arg1) {
+ return Integer.compare(arg1.get_error_time_secs(), arg0.get_error_time_secs());
+ }
+ });
return errorInfos;
}
http://git-wip-us.apache.org/repos/asf/storm/blob/b09e2755/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
index 56115ce..4cf0c05 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
@@ -53,6 +53,26 @@ public class ZKStateStorage implements IStateStorage {
private Map authConf;
private Map<Object, Object> conf;
+ private class ZkWatcherCallBack implements WatcherCallBack{
+ @Override
+ public void execute(Watcher.Event.KeeperState state, Watcher.Event.EventType type, String path) {
+ if (active.get()) {
+ if (!(state.equals(Watcher.Event.KeeperState.SyncConnected))) {
+ LOG.debug("Received event {} : {}: {} with disconnected Zookeeper.", state, type, path);
+ } else {
+ LOG.debug("Received event {} : {} : {}", state, type, path);
+ }
+
+ if (!type.equals(Watcher.Event.EventType.None)) {
+ for (Map.Entry<String, ZKStateChangedCallback> e : callbacks.entrySet()) {
+ ZKStateChangedCallback fn = e.getValue();
+ fn.changed(type, path);
+ }
+ }
+ }
+ }
+ }
+
public ZKStateStorage(Map<Object, Object> conf, Map authConf, List<ACL> acls, ClusterStateContext context) throws Exception {
this.conf = conf;
this.authConf = authConf;
@@ -66,45 +86,9 @@ public class ZKStateStorage implements IStateStorage {
zkTemp.close();
active = new AtomicBoolean(true);
- zkWriter = mkZk(new WatcherCallBack() {
- @Override
- public void execute(Watcher.Event.KeeperState state, Watcher.Event.EventType type, String path) {
- if (active.get()) {
- if (!(state.equals(Watcher.Event.KeeperState.SyncConnected))) {
- LOG.warn("Received event {} : {}: {} with disconnected Zookeeper.", state, type, path);
- } else {
- LOG.info("Received event {} : {} : {}", state, type, path);
- }
-
- if (!type.equals(Watcher.Event.EventType.None)) {
- for (Map.Entry<String, ZKStateChangedCallback> e : callbacks.entrySet()) {
- ZKStateChangedCallback fn = e.getValue();
- fn.changed(type, path);
- }
- }
- }
- }
- });
+ zkWriter = mkZk(new ZkWatcherCallBack());
if (isNimbus) {
- zkReader = mkZk(new WatcherCallBack() {
- @Override
- public void execute(Watcher.Event.KeeperState state, Watcher.Event.EventType type, String path) {
- if (active.get()) {
- if (!(state.equals(Watcher.Event.KeeperState.SyncConnected))) {
- LOG.warn("Received event {} : {}: {} with disconnected Zookeeper.", state, type, path);
- } else {
- LOG.debug("Received event {} : {} : {}", state, type, path);
- }
-
- if (!type.equals(Watcher.Event.EventType.None)) {
- for (Map.Entry<String, ZKStateChangedCallback> e : callbacks.entrySet()) {
- ZKStateChangedCallback fn = e.getValue();
- fn.changed(type, path);
- }
- }
- }
- }
- });
+ zkReader = mkZk(new ZkWatcherCallBack());
} else {
zkReader = zkWriter;
}
@@ -157,15 +141,15 @@ public class ZKStateStorage implements IStateStorage {
@Override
public void set_ephemeral_node(String path, byte[] data, List<ACL> acls) {
- Zookeeper.mkdirs(zkWriter, parentPath(path), acls);
+ Zookeeper.mkdirs(zkWriter, Zookeeper.parentPath(path), acls);
if (Zookeeper.exists(zkWriter, path, false)) {
try {
Zookeeper.setData(zkWriter, path, data);
- } catch (Exception e) {
+ } catch (RuntimeException e) {
if (Utils.exceptionCauseIsInstanceOf(KeeperException.NoNodeException.class, e)) {
Zookeeper.createNode(zkWriter, path, data, CreateMode.EPHEMERAL, acls);
} else {
- throw Utils.wrapInRuntime(e);
+ throw e;
}
}
@@ -182,7 +166,7 @@ public class ZKStateStorage implements IStateStorage {
@Override
public boolean node_exists(String path, boolean watch) {
- return Zookeeper.existsNode(zkWriter, path, watch);
+ return Zookeeper.existsNode(zkReader, path, watch);
}
@Override
@@ -204,7 +188,7 @@ public class ZKStateStorage implements IStateStorage {
if (Zookeeper.exists(zkWriter, path, false)) {
Zookeeper.setData(zkWriter, path, data);
} else {
- Zookeeper.mkdirs(zkWriter, parentPath(path), acls);
+ Zookeeper.mkdirs(zkWriter, Zookeeper.parentPath(path), acls);
Zookeeper.createNode(zkWriter, path, data, CreateMode.PERSISTENT, acls);
}
}
@@ -257,14 +241,4 @@ public class ZKStateStorage implements IStateStorage {
public void sync_path(String path) {
Zookeeper.syncPath(zkWriter, path);
}
-
- // To be remove when finished port Util.clj
- public static String parentPath(String path) {
- List<String> toks = Zookeeper.tokenizePath(path);
- int size = toks.size();
- if (size > 0) {
- toks.remove(size - 1);
- }
- return Zookeeper.toksToPath(toks);
- }
}
http://git-wip-us.apache.org/repos/asf/storm/blob/b09e2755/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java b/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
index e5b2666..5e9039a 100644
--- a/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
+++ b/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
@@ -394,9 +394,12 @@ public class Zookeeper {
}
public static String parentPath(String path) {
- List<String> tokens = tokenizePath(path);
- tokens.remove(tokens.size() - 1);
- return "/" + StringUtils.join(tokens, "/");
+ List<String> toks = Zookeeper.tokenizePath(path);
+ int size = toks.size();
+ if (size > 0) {
+ toks.remove(size - 1);
+ }
+ return Zookeeper.toksToPath(toks);
}
public static String toksToPath(List<String> toks) {
[14/27] storm git commit: Merge branch 'master' into ClusterUtils
Posted by bo...@apache.org.
Merge branch 'master' into ClusterUtils
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/924f9a29
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/924f9a29
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/924f9a29
Branch: refs/heads/master
Commit: 924f9a29dc1713921178e61ce2f5b6a14df97ed7
Parents: 2ee8bec 12ceb09
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Mon Feb 15 15:13:33 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Tue Feb 16 11:14:18 2016 +0800
----------------------------------------------------------------------
CHANGELOG.md | 16 +
README.markdown | 1 +
bin/storm-config.cmd | 6 +-
bin/storm.cmd | 47 +-
bin/storm.py | 8 +-
conf/defaults.yaml | 4 +
dev-tools/travis/travis-script.sh | 4 +-
.../starter/trident/TridentMapExample.java | 123 +++
external/sql/storm-sql-core/pom.xml | 9 +
external/storm-elasticsearch/pom.xml | 2 +
.../storm/hbase/security/HBaseSecurityUtil.java | 36 +-
.../jvm/org/apache/storm/kafka/KafkaSpout.java | 8 +-
.../jvm/org/apache/storm/kafka/KafkaUtils.java | 44 +-
.../apache/storm/kafka/PartitionManager.java | 42 +-
.../kafka/trident/TridentKafkaEmitter.java | 23 +-
external/storm-mqtt/core/pom.xml | 4 +-
log4j2/cluster.xml | 2 +-
log4j2/worker.xml | 2 +-
pom.xml | 9 +-
storm-core/pom.xml | 11 +-
.../src/clj/org/apache/storm/LocalCluster.clj | 4 +-
storm-core/src/clj/org/apache/storm/clojure.clj | 8 +-
.../clj/org/apache/storm/command/blobstore.clj | 11 +-
.../org/apache/storm/command/config_value.clj | 25 -
.../org/apache/storm/command/dev_zookeeper.clj | 6 +-
.../clj/org/apache/storm/command/get_errors.clj | 12 +-
.../apache/storm/command/shell_submission.clj | 4 +-
storm-core/src/clj/org/apache/storm/config.clj | 18 +-
.../src/clj/org/apache/storm/converter.clj | 14 +-
.../src/clj/org/apache/storm/daemon/acker.clj | 21 +-
.../src/clj/org/apache/storm/daemon/common.clj | 46 +-
.../src/clj/org/apache/storm/daemon/drpc.clj | 25 +-
.../clj/org/apache/storm/daemon/executor.clj | 530 +++++-----
.../clj/org/apache/storm/daemon/logviewer.clj | 70 +-
.../src/clj/org/apache/storm/daemon/nimbus.clj | 172 ++--
.../clj/org/apache/storm/daemon/supervisor.clj | 202 ++--
.../src/clj/org/apache/storm/daemon/task.clj | 2 +-
.../src/clj/org/apache/storm/daemon/worker.clj | 78 +-
.../src/clj/org/apache/storm/disruptor.clj | 10 +-
storm-core/src/clj/org/apache/storm/event.clj | 2 +-
.../src/clj/org/apache/storm/local_state.clj | 9 +-
.../clj/org/apache/storm/messaging/loader.clj | 34 -
.../clj/org/apache/storm/messaging/local.clj | 23 -
.../org/apache/storm/pacemaker/pacemaker.clj | 7 +-
.../storm/pacemaker/pacemaker_state_factory.clj | 122 ---
.../clj/org/apache/storm/process_simulator.clj | 4 +-
.../apache/storm/scheduler/DefaultScheduler.clj | 7 +-
.../apache/storm/scheduler/EvenScheduler.clj | 23 +-
.../storm/scheduler/IsolationScheduler.clj | 29 +-
storm-core/src/clj/org/apache/storm/stats.clj | 82 +-
storm-core/src/clj/org/apache/storm/testing.clj | 89 +-
storm-core/src/clj/org/apache/storm/thrift.clj | 6 +-
storm-core/src/clj/org/apache/storm/timer.clj | 12 +-
.../clj/org/apache/storm/trident/testing.clj | 9 +-
storm-core/src/clj/org/apache/storm/ui/core.clj | 99 +-
.../src/clj/org/apache/storm/ui/helpers.clj | 14 +-
storm-core/src/clj/org/apache/storm/util.clj | 925 +----------------
storm-core/src/jvm/org/apache/storm/Config.java | 39 +
.../org/apache/storm/cluster/ClusterUtils.java | 49 +-
.../storm/cluster/IStormClusterState.java | 9 +-
.../storm/cluster/PaceMakerStateStorage.java | 4 +-
.../storm/cluster/StateStorageFactory.java | 2 +-
.../storm/cluster/StormClusterStateImpl.java | 51 +-
.../storm/cluster/ZKStateStorageFactory.java | 4 +-
.../org/apache/storm/command/ConfigValue.java | 30 +
.../storm/daemon/metrics/MetricsUtils.java | 108 ++
.../reporters/ConsolePreparableReporter.java | 76 ++
.../reporters/CsvPreparableReporter.java | 80 ++
.../reporters/JmxPreparableReporter.java | 70 ++
.../metrics/reporters/PreparableReporter.java | 32 +
.../storm/logging/ThriftAccessLogger.java | 13 +-
.../apache/storm/pacemaker/PacemakerClient.java | 5 +
.../serialization/SerializationFactory.java | 17 +-
.../staticmocking/MockedConfigUtils.java | 31 -
.../jvm/org/apache/storm/trident/Stream.java | 87 +-
.../storm/trident/operation/Consumer.java | 35 +
.../trident/operation/FlatMapFunction.java | 37 +
.../storm/trident/operation/MapFunction.java | 36 +
.../operation/impl/ConsumerExecutor.java | 38 +
.../operation/impl/FlatMapFunctionExecutor.java | 43 +
.../operation/impl/MapFunctionExecutor.java | 41 +
.../trident/planner/processor/MapProcessor.java | 87 ++
.../jvm/org/apache/storm/utils/ConfigUtils.java | 20 +-
.../jvm/org/apache/storm/utils/Container.java | 11 +-
.../jvm/org/apache/storm/utils/IPredicate.java | 27 +
.../org/apache/storm/utils/NimbusClient.java | 2 +-
.../utils/StormConnectionStateConverter.java | 44 +
.../jvm/org/apache/storm/utils/TestUtils.java | 34 -
.../src/jvm/org/apache/storm/utils/Time.java | 26 +-
.../src/jvm/org/apache/storm/utils/Utils.java | 989 +++++++++++++++++--
.../storm/validation/ConfigValidation.java | 2 +-
.../org/apache/storm/zookeeper/Zookeeper.java | 7 +
.../org/apache/storm/integration_test.clj | 98 +-
.../org/apache/storm/testing4j_test.clj | 37 +-
.../apache/storm/trident/integration_test.clj | 15 +-
.../test/clj/org/apache/storm/cluster_test.clj | 27 +-
.../test/clj/org/apache/storm/drpc_test.clj | 23 +-
.../clj/org/apache/storm/logviewer_test.clj | 267 ++---
.../storm/messaging/netty_integration_test.clj | 2 +-
.../test/clj/org/apache/storm/nimbus_test.clj | 112 ++-
.../storm/pacemaker_state_factory_test.clj | 74 +-
.../scheduler/resource_aware_scheduler_test.clj | 21 +-
.../apache/storm/security/auth/auth_test.clj | 11 +-
.../authorizer/DRPCSimpleACLAuthorizer_test.clj | 2 +-
.../BlowfishTupleSerializer_test.clj | 1 -
.../clj/org/apache/storm/serialization_test.clj | 23 +-
.../clj/org/apache/storm/supervisor_test.clj | 649 ++++++------
.../clj/org/apache/storm/transactional_test.clj | 18 +
.../clj/org/apache/storm/trident/state_test.clj | 5 +-
.../clj/org/apache/storm/trident/tuple_test.clj | 15 +-
.../test/clj/org/apache/storm/utils_test.clj | 16 +-
.../test/clj/org/apache/storm/worker_test.clj | 1 -
.../staticmocking/ConfigUtilsInstaller.java | 38 +
.../utils/staticmocking/UtilsInstaller.java | 38 +
.../storm/utils/staticmocking/package-info.java | 95 ++
115 files changed, 4281 insertions(+), 2648 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/conf/defaults.yaml
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
index 7be526d,657e242..ab5cbed
--- a/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
+++ b/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
@@@ -14,7 -14,8 +14,8 @@@
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns org.apache.storm.command.dev-zookeeper
+ (:import [org.apache.storm.utils Utils])
- (:use [org.apache.storm zookeeper util config])
+ (:use [org.apache.storm util config])
(:import [org.apache.storm.utils ConfigUtils])
(:import [org.apache.storm.zookeeper Zookeeper])
(:gen-class))
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/command/shell_submission.clj
index 3978d2f,0253338..870e7f6
--- a/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
+++ b/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
@@@ -15,8 -15,9 +15,9 @@@
;; limitations under the License.
(ns org.apache.storm.command.shell-submission
(:import [org.apache.storm StormSubmitter]
+ [org.apache.storm.utils Utils]
[org.apache.storm.zookeeper Zookeeper])
- (:use [org.apache.storm thrift util config log zookeeper])
+ (:use [org.apache.storm thrift util config log])
(:require [clojure.string :as str])
(:import [org.apache.storm.utils ConfigUtils])
(:gen-class))
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/converter.clj
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/daemon/common.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/common.clj
index b144f40,eb1ec1e..db342d2
--- a/storm-core/src/clj/org/apache/storm/daemon/common.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/common.clj
@@@ -15,17 -15,19 +15,20 @@@
;; limitations under the License.
(ns org.apache.storm.daemon.common
(:use [org.apache.storm log config util])
- (:import [org.apache.storm.generated StormTopology
+ (:import [org.apache.storm.generated StormTopology NodeInfo
InvalidTopologyException GlobalStreamId]
- [org.apache.storm.utils ThriftTopologyUtils])
- (:import [org.apache.storm.utils Utils ConfigUtils])
+ [org.apache.storm.utils Utils ConfigUtils IPredicate ThriftTopologyUtils]
+ [org.apache.storm.daemon.metrics.reporters PreparableReporter]
+ [com.codahale.metrics MetricRegistry])
+ (:import [org.apache.storm.daemon.metrics MetricsUtils])
(:import [org.apache.storm.task WorkerTopologyContext])
(:import [org.apache.storm Constants])
+ (:import [org.apache.storm.cluster StormClusterStateImpl])
(:import [org.apache.storm.metric SystemBolt])
(:import [org.apache.storm.metric EventLoggerBolt])
- (:import [org.apache.storm.security.auth IAuthorizer])
- (:import [java.io InterruptedIOException])
+ (:import [org.apache.storm.security.auth IAuthorizer])
+ (:import [java.io InterruptedIOException]
+ [org.json.simple JSONValue])
(:require [clojure.set :as set])
(:require [org.apache.storm.daemon.acker :as acker])
(:require [org.apache.storm.thrift :as thrift])
@@@ -73,12 -83,10 +84,11 @@@
(defn new-executor-stats []
(ExecutorStats. 0 0 0 0 0))
+
(defn get-storm-id [storm-cluster-state storm-name]
- (let [active-storms (.activeStorms storm-cluster-state)]
- (find-first
- #(= storm-name (.get_name (.stormBase storm-cluster-state % nil)))
- active-storms)
- (let [active-storms (.active-storms storm-cluster-state)
- pred (reify IPredicate (test [this x] (= storm-name (:storm-name (.storm-base storm-cluster-state x nil)))))]
++ (let [active-storms (.activeStorms storm-cluster-state)
++ pred (reify IPredicate (test [this x] (= storm-name (.get_name (.stormBase storm-cluster-state x nil)))))]
+ (Utils/findOne pred active-storms)
))
(defn topology-bases [storm-cluster-state]
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/executor.clj
index 49ae6cf,e2380b7..33b89ed
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@@ -34,10 -34,13 +34,12 @@@
(:import [org.apache.storm.daemon Shutdownable])
(:import [org.apache.storm.metric.api IMetric IMetricsConsumer$TaskInfo IMetricsConsumer$DataPoint StateMetric])
(:import [org.apache.storm Config Constants])
- (:import [org.apache.storm.cluster ClusterStateContext DaemonType])
+ (:import [org.apache.storm.cluster ClusterStateContext DaemonType StormClusterStateImpl ClusterUtils])
(:import [org.apache.storm.grouping LoadAwareCustomStreamGrouping LoadAwareShuffleGrouping LoadMapping ShuffleGrouping])
- (:import [java.util.concurrent ConcurrentLinkedQueue])
+ (:import [java.lang Thread Thread$UncaughtExceptionHandler]
+ [java.util.concurrent ConcurrentLinkedQueue]
+ [org.json.simple JSONValue])
- (:require [org.apache.storm [thrift :as thrift]
- [cluster :as cluster] [disruptor :as disruptor] [stats :as stats]])
+ (:require [org.apache.storm [thrift :as thrift] [disruptor :as disruptor] [stats :as stats]])
(:require [org.apache.storm.daemon [task :as task]])
(:require [org.apache.storm.daemon.builtin-metrics :as builtin-metrics])
(:require [clojure.set :as set]))
@@@ -206,9 -211,9 +210,9 @@@
(swap! interval-errors inc)
(when (<= @interval-errors max-per-interval)
- (cluster/report-error (:storm-cluster-state executor) (:storm-id executor) (:component-id executor)
+ (.reportError (:storm-cluster-state executor) (:storm-id executor) (:component-id executor)
- (hostname storm-conf)
+ (Utils/hostname storm-conf)
- (.getThisWorkerPort (:worker-context executor)) error)
+ (long (.getThisWorkerPort (:worker-context executor))) error)
))))
;; in its own function so that it can be mocked out by tracked topologies
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index daf5e45,710cd83..6bdbdc0
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@@ -38,9 -39,9 +39,9 @@@
(:import [org.apache.storm.scheduler INimbus SupervisorDetails WorkerSlot TopologyDetails
Cluster Topologies SchedulerAssignment SchedulerAssignmentImpl DefaultScheduler ExecutorDetails])
(:import [org.apache.storm.nimbus NimbusInfo])
- (:import [org.apache.storm.utils TimeCacheMap TimeCacheMap$ExpiredCallback Utils ConfigUtils TupleUtils ThriftTopologyUtils
+ (:import [org.apache.storm.utils TimeCacheMap Time TimeCacheMap$ExpiredCallback Utils ConfigUtils TupleUtils ThriftTopologyUtils
BufferFileInputStream BufferInputStream])
- (:import [org.apache.storm.generated NotAliveException AlreadyAliveException StormTopology ErrorInfo
+ (:import [org.apache.storm.generated NotAliveException AlreadyAliveException StormTopology ErrorInfo ClusterWorkerHeartbeat
ExecutorInfo InvalidTopologyException Nimbus$Iface Nimbus$Processor SubmitOptions TopologyInitialStatus
KillOptions RebalanceOptions ClusterSummary SupervisorSummary TopologySummary TopologyInfo TopologyHistoryInfo
ExecutorSummary AuthorizationException GetInfoOptions NumErrorsChoice SettableBlobMeta ReadableBlobMeta
@@@ -407,10 -414,10 +409,10 @@@
[storm-cluster-state]
(let [assignments (.assignments storm-cluster-state nil)]
- (defaulted
+ (or
(apply merge-with set/union
(for [a assignments
- [_ [node port]] (-> (.assignment-info storm-cluster-state a nil) :executor->node+port)]
+ [_ [node port]] (-> (clojurify-assignment (.assignmentInfo storm-cluster-state a nil)) :executor->node+port)]
{node #{port}}
))
{})
@@@ -586,12 -594,7 +589,12 @@@
(defn update-heartbeats! [nimbus storm-id all-executors existing-assignment]
(log-debug "Updating heartbeats for " storm-id " " (pr-str all-executors))
(let [storm-cluster-state (:storm-cluster-state nimbus)
- executor-beats (.executor-beats storm-cluster-state storm-id (:executor->node+port existing-assignment))
+ executor-beats (let [executor-stats-java-map (.executorBeats storm-cluster-state storm-id (.get_executor_node_port (thriftify-assignment existing-assignment)))
+ executor-stats-clojurify (clojurify-structure executor-stats-java-map)]
- (->> (dofor [[^ExecutorInfo executor-info ^ClusterWorkerHeartbeat cluster-worker-heartbeat] executor-stats-clojurify]
- {[(.get_task_start executor-info) (.get_task_end executor-info)] (clojurify-zk-worker-hb cluster-worker-heartbeat)})
++ (->> (dofor [[^ExecutorInfo executor-info executor-heartbeat] executor-stats-clojurify]
++ {[(.get_task_start executor-info) (.get_task_end executor-info)] executor-heartbeat})
+ (apply merge)))
+
cache (update-heartbeat-cache (@(:heartbeats-cache nimbus) storm-id)
executor-beats
all-executors
@@@ -988,10 -1002,10 +1002,10 @@@
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
+ (.activateStorm storm-cluster-state
storm-id
- (StormBase. storm-name
+ (thriftify-storm-base (StormBase. storm-name
- (current-time-secs)
+ (Time/currentTimeSecs)
{:type topology-initial-status}
(storm-conf TOPOLOGY-WORKERS)
num-executors
@@@ -1137,9 -1152,9 +1152,9 @@@
(when-not (empty? to-cleanup-ids)
(doseq [id to-cleanup-ids]
(log-message "Cleaning up " id)
- (.teardown-heartbeats! storm-cluster-state id)
- (.teardown-topology-errors! storm-cluster-state id)
+ (.teardownHeartbeats storm-cluster-state id)
+ (.teardownTopologyErrors storm-cluster-state id)
- (rmr (ConfigUtils/masterStormDistRoot conf id))
+ (Utils/forceDelete (ConfigUtils/masterStormDistRoot conf id))
(blob-rm-topology-keys id blob-store storm-cluster-state)
(swap! (:heartbeats-cache nimbus) dissoc id)))))
(log-message "not a leader, skipping cleanup")))
@@@ -1811,8 -1830,8 +1838,8 @@@
(.set_used_cpu sup-sum used-cpu))
(when-let [version (:version info)] (.set_version sup-sum version))
sup-sum))
- nimbus-uptime ((:uptime nimbus))
+ nimbus-uptime (. (:uptime nimbus) upTime)
- bases (topology-bases storm-cluster-state)
+ bases (nimbus-topology-bases storm-cluster-state)
nimbuses (.nimbuses storm-cluster-state)
;;update the isLeader field for each nimbus summary
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
index 3a83d03,ae9e92f..273a6bd
--- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
@@@ -16,14 -16,15 +16,15 @@@
(ns org.apache.storm.daemon.supervisor
(:import [java.io File IOException FileOutputStream])
(:import [org.apache.storm.scheduler ISupervisor]
- [org.apache.storm.utils LocalState Time Utils ConfigUtils]
+ [org.apache.storm.utils LocalState Time Utils Utils$ExitCodeCallable
+ ConfigUtils]
[org.apache.storm.daemon Shutdownable]
[org.apache.storm Constants]
- [org.apache.storm.cluster ClusterStateContext DaemonType]
+ [org.apache.storm.cluster ClusterStateContext DaemonType StormClusterStateImpl ClusterUtils]
[java.net JarURLConnection]
- [java.net URI]
+ [java.net URI URLDecoder]
[org.apache.commons.io FileUtils])
- (:use [org.apache.storm config util log timer local-state])
+ (:use [org.apache.storm config util log timer local-state converter])
(:import [org.apache.storm.generated AuthorizationException KeyNotFoundException WorkerResources])
(:import [org.apache.storm.utils NimbusLeaderNotFoundException VersionInfo])
(:import [java.nio.file Files StandardCopyOption])
@@@ -315,12 -319,14 +321,12 @@@
:shared-context shared-context
:isupervisor isupervisor
:active (atom true)
- :uptime (uptime-computer)
+ :uptime (Utils/makeUptimeComputer)
:version STORM-VERSION
:worker-thread-pids-atom (atom {})
- :storm-cluster-state (cluster/mk-storm-cluster-state conf :acls (when
- (Utils/isZkAuthenticationConfiguredStormServer
- conf)
- SUPERVISOR-ZK-ACLS)
- :context (ClusterStateContext. DaemonType/SUPERVISOR))
+ :storm-cluster-state (ClusterUtils/mkStormClusterState conf (when (Utils/isZkAuthenticationConfiguredStormServer conf)
+ SUPERVISOR-ZK-ACLS)
+ (ClusterStateContext. DaemonType/SUPERVISOR))
:local-state (ConfigUtils/supervisorState conf)
:supervisor-id (.getSupervisorId isupervisor)
:assignment-id (.getAssignmentId isupervisor)
@@@ -777,19 -791,19 +792,19 @@@
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!
+ heartbeat-fn (fn [] (.supervisorHeartbeat
(:storm-cluster-state supervisor)
(:supervisor-id supervisor)
- (thriftify-supervisor-info (->SupervisorInfo (current-time-secs)
- (->SupervisorInfo (Time/currentTimeSecs)
++ (thriftify-supervisor-info (->SupervisorInfo (Time/currentTimeSecs)
(:my-hostname supervisor)
(:assignment-id supervisor)
(keys @(:curr-assignment supervisor))
;; used ports
(.getMetadata isupervisor)
(conf SUPERVISOR-SCHEDULER-META)
- ((:uptime supervisor))
+ (. (:uptime supervisor) upTime)
(:version supervisor)
- (mk-supervisor-capacities conf))))]
+ (mk-supervisor-capacities conf)))))]
(heartbeat-fn)
;; should synchronize supervisor so it doesn't launch anything after being down (optimization)
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/worker.clj
index ae5be57,fe8cfae..9863427
--- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
@@@ -19,16 -19,18 +19,18 @@@
(:require [clj-time.core :as time])
(:require [clj-time.coerce :as coerce])
(:require [org.apache.storm.daemon [executor :as executor]])
- (:require [org.apache.storm [disruptor :as disruptor] [cluster :as cluster]])
+ (:require [org.apache.storm [disruptor :as disruptor]])
(:require [clojure.set :as set])
- (:require [org.apache.storm.messaging.loader :as msg-loader])
(:import [java.util.concurrent Executors]
- [org.apache.storm.hooks IWorkerHook BaseWorkerHook])
- (:import [java.util ArrayList HashMap])
- (:import [org.apache.storm.utils Utils ConfigUtils TransferDrainer ThriftTopologyUtils WorkerBackpressureThread DisruptorQueue])
+ [org.apache.storm.hooks IWorkerHook BaseWorkerHook]
+ [uk.org.lidalia.sysoutslf4j.context SysOutOverSLF4J])
+ (:import [java.util ArrayList HashMap]
+ [java.util.concurrent.locks ReentrantReadWriteLock])
+ (:import [org.apache.commons.io FileUtils])
+ (:import [org.apache.storm.utils Utils ConfigUtils TransferDrainer ThriftTopologyUtils WorkerBackpressureThread DisruptorQueue Time])
(:import [org.apache.storm.grouping LoadMapping])
(:import [org.apache.storm.messaging TransportFactory])
- (:import [org.apache.storm.messaging TaskMessage IContext IConnection ConnectionWithStatus ConnectionWithStatus$Status])
+ (:import [org.apache.storm.messaging TaskMessage IContext IConnection ConnectionWithStatus ConnectionWithStatus$Status DeserializingConnectionCallback])
(:import [org.apache.storm.daemon Shutdownable])
(:import [org.apache.storm.serialization KryoTupleSerializer])
(:import [org.apache.storm.generated StormTopology])
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
index a36da3a,be4361a..0000000
deleted file mode 100644,100644
--- a/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
+++ /dev/null
@@@ -1,122 -1,141 +1,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.
--
--(ns org.apache.storm.pacemaker.pacemaker-state-factory
-- (:require [org.apache.storm.pacemaker pacemaker]
- [org.apache.storm.cluster-state [zookeeper-state-factory :as zk-factory]]
-- [org.apache.storm
-- [config :refer :all]
- [cluster :refer :all]
-- [log :refer :all]
-- [util :as util]])
-- (:import [org.apache.storm.generated
-- HBExecutionException HBServerMessageType HBMessage
-- HBMessageData HBPulse]
- [org.apache.storm.cluster ZKStateStorage ClusterUtils IStateStorage]
- [org.apache.storm.cluster_state zookeeper_state_factory]
- [org.apache.storm.cluster ClusterState]
-- [org.apache.storm.pacemaker PacemakerClient])
-- (:gen-class
- :implements [org.apache.storm.cluster.StateStorageFactory]))
- :implements [org.apache.storm.cluster.ClusterStateFactory]))
--
--;; So we can mock the client for testing
--(defn makeClient [conf]
-- (PacemakerClient. conf))
--
--(defn makeZKState [conf auth-conf acls context]
- (ClusterUtils/mkStateStorage conf auth-conf acls context))
- (.mkState (zookeeper_state_factory.) conf auth-conf acls context))
--
--(def max-retries 10)
-
-(defn retry-on-exception
- "Retries specific function on exception based on retries count"
- [retries task-description f & args]
- (let [res (try {:value (apply f args)}
- (catch Exception e
- (if (<= 0 retries)
- (throw e)
- {:exception e})))]
- (if (:exception res)
- (do
- (log-error (:exception res) (str "Failed to " task-description ". Will make [" retries "] more attempts."))
- (recur (dec retries) task-description f args))
- (do
- (log-debug (str "Successful " task-description "."))
- (:value res)))))
--
--(defn -mkState [this conf auth-conf acls context]
-- (let [zk-state (makeZKState conf auth-conf acls context)
-- pacemaker-client (makeClient conf)]
--
-- (reify
- IStateStorage
- ClusterState
-- ;; Let these pass through to the zk-state. We only want to handle heartbeats.
-- (register [this callback] (.register zk-state callback))
-- (unregister [this callback] (.unregister zk-state callback))
-- (set_ephemeral_node [this path data acls] (.set_ephemeral_node zk-state path data acls))
-- (create_sequential [this path data acls] (.create_sequential zk-state path data acls))
-- (set_data [this path data acls] (.set_data zk-state path data acls))
-- (delete_node [this path] (.delete_node zk-state path))
-- (delete_node_blobstore [this path nimbus-host-port-info] (.delete_node_blobstore zk-state path nimbus-host-port-info))
-- (get_data [this path watch?] (.get_data zk-state path watch?))
-- (get_data_with_version [this path watch?] (.get_data_with_version zk-state path watch?))
-- (get_version [this path watch?] (.get_version zk-state path watch?))
-- (get_children [this path watch?] (.get_children zk-state path watch?))
-- (mkdirs [this path acls] (.mkdirs zk-state path acls))
-- (node_exists [this path watch?] (.node_exists zk-state path watch?))
-- (add_listener [this listener] (.add_listener zk-state listener))
-- (sync_path [this path] (.sync_path zk-state path))
--
-- (set_worker_hb [this path data acls]
- (util/retry-on-exception
- (retry-on-exception
-- max-retries
-- "set_worker_hb"
-- #(let [response
-- (.send pacemaker-client
-- (HBMessage. HBServerMessageType/SEND_PULSE
-- (HBMessageData/pulse
-- (doto (HBPulse.)
-- (.set_id path)
-- (.set_details data)))))]
-- (if (= (.get_type response) HBServerMessageType/SEND_PULSE_RESPONSE)
-- :ok
-- (throw (HBExecutionException. "Invalid Response Type"))))))
--
-- (delete_worker_hb [this path]
- (util/retry-on-exception
- (retry-on-exception
-- max-retries
-- "delete_worker_hb"
-- #(let [response
-- (.send pacemaker-client
-- (HBMessage. HBServerMessageType/DELETE_PATH
-- (HBMessageData/path path)))]
-- (if (= (.get_type response) HBServerMessageType/DELETE_PATH_RESPONSE)
-- :ok
-- (throw (HBExecutionException. "Invalid Response Type"))))))
--
-- (get_worker_hb [this path watch?]
- (util/retry-on-exception
- (retry-on-exception
-- max-retries
-- "get_worker_hb"
-- #(let [response
-- (.send pacemaker-client
-- (HBMessage. HBServerMessageType/GET_PULSE
-- (HBMessageData/path path)))]
-- (if (= (.get_type response) HBServerMessageType/GET_PULSE_RESPONSE)
-- (try
-- (.get_details (.get_pulse (.get_data response)))
-- (catch Exception e
-- (throw (HBExecutionException. (.toString e)))))
-- (throw (HBExecutionException. "Invalid Response Type"))))))
--
-- (get_worker_hb_children [this path watch?]
- (util/retry-on-exception
- (retry-on-exception
-- max-retries
-- "get_worker_hb_children"
-- #(let [response
-- (.send pacemaker-client
-- (HBMessage. HBServerMessageType/GET_ALL_NODES_FOR_PATH
-- (HBMessageData/path path)))]
-- (if (= (.get_type response) HBServerMessageType/GET_ALL_NODES_FOR_PATH_RESPONSE)
-- (try
-- (into [] (.get_pulseIds (.get_nodes (.get_data response))))
-- (catch Exception e
-- (throw (HBExecutionException. (.toString e)))))
-- (throw (HBExecutionException. "Invalid Response Type"))))))
--
-- (close [this]
-- (.close zk-state)
-- (.close pacemaker-client)))))
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/stats.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/stats.clj
index 0bf1757,8b37fc3..8632ed3
--- a/storm-core/src/clj/org/apache/storm/stats.clj
+++ b/storm-core/src/clj/org/apache/storm/stats.clj
@@@ -24,8 -24,8 +24,9 @@@
ExecutorAggregateStats SpecificAggregateStats
SpoutAggregateStats TopologyPageInfo TopologyStats])
(:import [org.apache.storm.utils Utils])
+ (:import [org.apache.storm.cluster StormClusterStateImpl])
- (:import [org.apache.storm.metric.internal MultiCountStatAndMetric MultiLatencyStatAndMetric])
+ (:import [org.apache.storm.metric.internal MultiCountStatAndMetric MultiLatencyStatAndMetric]
+ [java.util Collection])
(:use [org.apache.storm log util])
(:use [clojure.math.numeric-tower :only [ceil]]))
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/testing.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/testing.clj
index 5a0bdf2,c872742..eef7754
--- a/storm-core/src/clj/org/apache/storm/testing.clj
+++ b/storm-core/src/clj/org/apache/storm/testing.clj
@@@ -44,11 -45,12 +45,12 @@@
(:import [org.apache.storm.transactional.partitioned PartitionedTransactionalSpoutExecutor])
(:import [org.apache.storm.tuple Tuple])
(:import [org.apache.storm.generated StormTopology])
- (:import [org.apache.storm.task TopologyContext])
+ (:import [org.apache.storm.task TopologyContext]
+ (org.apache.storm.messaging IContext)
+ [org.json.simple JSONValue])
- (:require [org.apache.storm [zookeeper :as zk]])
+ (:import [org.apache.storm.cluster ZKStateStorage ClusterStateContext StormClusterStateImpl ClusterUtils])
- (:require [org.apache.storm.messaging.loader :as msg-loader])
(:require [org.apache.storm.daemon.acker :as acker])
- (:use [org.apache.storm cluster util thrift config log local-state]))
+ (:use [org.apache.storm util thrift config log local-state converter]))
(defn feeder-spout
[fields]
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/thrift.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/thrift.clj
index 4dc21f9,779c1d1..7aab729
--- a/storm-core/src/clj/org/apache/storm/thrift.clj
+++ b/storm-core/src/clj/org/apache/storm/thrift.clj
@@@ -29,8 -29,9 +29,9 @@@
(:import [org.apache.storm.grouping CustomStreamGrouping])
(:import [org.apache.storm.topology TopologyBuilder])
(:import [org.apache.storm.clojure RichShellBolt RichShellSpout])
- (:import [org.apache.thrift.transport TTransport])
+ (:import [org.apache.thrift.transport TTransport]
+ (org.json.simple JSONValue))
- (:use [org.apache.storm util config log zookeeper]))
+ (:use [org.apache.storm util config log]))
(defn instantiate-java-object
[^JavaObject obj]
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/ui/core.clj
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/util.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/util.clj
index 165d8ee,f685d12..72778bb
--- a/storm-core/src/clj/org/apache/storm/util.clj
+++ b/storm-core/src/clj/org/apache/storm/util.clj
@@@ -20,9 -20,8 +20,9 @@@
(:import [java.io FileReader FileNotFoundException])
(:import [java.nio.file Paths])
(:import [org.apache.storm Config])
- (:import [org.apache.storm.generated ErrorInfo])
- (:import [org.apache.storm.utils Time Container ClojureTimerTask Utils
- MutableObject MutableInt])
++ (:import [org.apache.storm.generated ErrorInfo])
+ (:import [org.apache.storm.utils Time ClojureTimerTask Utils
+ MutableObject])
(:import [org.apache.storm.security.auth NimbusPrincipal])
(:import [javax.security.auth Subject])
(:import [java.util UUID Random ArrayList List Collections])
@@@ -262,58 -163,9 +164,19 @@@
(instance? Boolean x) (boolean x)
true x))
s))
+; move this func form convert.clj due to cyclic load dependency
+(defn clojurify-error [^ErrorInfo error]
+ (if error
+ {
+ :error (.get_error error)
+ :time-secs (.get_error_time_secs error)
+ :host (.get_host error)
+ :port (.get_port error)
+ }
+ ))
- (defmacro with-file-lock
- [path & body]
- `(let [f# (File. ~path)
- _# (.createNewFile f#)
- rf# (RandomAccessFile. f# "rw")
- lock# (.. rf# (getChannel) (lock))]
- (try
- ~@body
- (finally
- (.release lock#)
- (.close rf#)))))
-
- (defn tokenize-path
- [^String path]
- (let [toks (.split path "/")]
- (vec (filter (complement empty?) toks))))
-
- (defn assoc-conj
- [m k v]
- (merge-with concat m {k [v]}))
-
- ;; returns [ones in first set not in second, ones in second set not in first]
- (defn set-delta
- [old curr]
- (let [s1 (set old)
- s2 (set curr)]
- [(set/difference s1 s2) (set/difference s2 s1)]))
-
- (defn parent-path
- [path]
- (let [toks (tokenize-path path)]
- (str "/" (str/join "/" (butlast toks)))))
-
- (defn toks->path
- [toks]
- (str "/" (str/join "/" toks)))
-
- (defn normalize-path
- [^String path]
- (toks->path (tokenize-path path)))
-
+ ;TODO: We're keeping this function around until all the code using it is properly tranlated to java
+ ;TODO: by properly having the for loop IN THE JAVA FUNCTION that originally used this function.
(defn map-val
[afn amap]
(into {}
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
index b30d1d2,0000000..0c663f0
mode 100644,000000..100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
@@@ -1,249 -1,0 +1,282 @@@
+/**
+ * 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.PersistentArrayMap;
++import clojure.lang.RT;
+import org.apache.storm.Config;
+import org.apache.storm.generated.ClusterWorkerHeartbeat;
+import org.apache.storm.generated.ExecutorInfo;
+import org.apache.storm.generated.ExecutorStats;
+import org.apache.storm.generated.ProfileAction;
+import org.apache.storm.utils.Utils;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
+import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
+
++import java.io.PrintWriter;
++import java.io.StringWriter;
+import java.io.UnsupportedEncodingException;
+import java.net.URLEncoder;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class ClusterUtils {
+
+ public static final String ZK_SEPERATOR = "/";
+
+ public static final String ASSIGNMENTS_ROOT = "assignments";
+ public static final String CODE_ROOT = "code";
+ public static final String STORMS_ROOT = "storms";
+ public static final String SUPERVISORS_ROOT = "supervisors";
+ public static final String WORKERBEATS_ROOT = "workerbeats";
+ public static final String BACKPRESSURE_ROOT = "backpressure";
+ public static final String ERRORS_ROOT = "errors";
+ public static final String BLOBSTORE_ROOT = "blobstore";
+ public static final String BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_ROOT = "blobstoremaxkeysequencenumber";
+ public static final String NIMBUSES_ROOT = "nimbuses";
+ public static final String CREDENTIALS_ROOT = "credentials";
+ public static final String LOGCONFIG_ROOT = "logconfigs";
+ public static final String PROFILERCONFIG_ROOT = "profilerconfigs";
+
+ public static final String ASSIGNMENTS_SUBTREE = ZK_SEPERATOR + ASSIGNMENTS_ROOT;
+ public static final String STORMS_SUBTREE = ZK_SEPERATOR + STORMS_ROOT;
+ public static final String SUPERVISORS_SUBTREE = ZK_SEPERATOR + SUPERVISORS_ROOT;
+ public static final String WORKERBEATS_SUBTREE = ZK_SEPERATOR + WORKERBEATS_ROOT;
+ public static final String BACKPRESSURE_SUBTREE = ZK_SEPERATOR + BACKPRESSURE_ROOT;
+ public static final String ERRORS_SUBTREE = ZK_SEPERATOR + ERRORS_ROOT;
+ public static final String BLOBSTORE_SUBTREE = ZK_SEPERATOR + BLOBSTORE_ROOT;
+ public static final String BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_SUBTREE = ZK_SEPERATOR + BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_ROOT;
+ public static final String NIMBUSES_SUBTREE = ZK_SEPERATOR + NIMBUSES_ROOT;
+ public static final String CREDENTIALS_SUBTREE = ZK_SEPERATOR + CREDENTIALS_ROOT;
+ public static final String LOGCONFIG_SUBTREE = ZK_SEPERATOR + LOGCONFIG_ROOT;
+ public static final String PROFILERCONFIG_SUBTREE = ZK_SEPERATOR + PROFILERCONFIG_ROOT;
+
+ // A singleton instance allows us to mock delegated static methods in our
+ // tests by subclassing.
+ private static final ClusterUtils INSTANCE = new ClusterUtils();
+ private static ClusterUtils _instance = INSTANCE;
+
+ /**
+ * Provide an instance of this class for delegates to use. To mock out delegated methods, provide an instance of a subclass that overrides the
+ * implementation of the delegated method.
+ *
+ * @param u a Cluster instance
+ */
+ public static void setInstance(ClusterUtils u) {
+ _instance = u;
+ }
+
+ /**
+ * Resets the singleton instance to the default. This is helpful to reset the class to its original functionality when mocking is no longer desired.
+ */
+ public static void resetInstance() {
+ _instance = INSTANCE;
+ }
+
+ public static List<ACL> mkTopoOnlyAcls(Map topoConf) throws NoSuchAlgorithmException {
+ List<ACL> aclList = null;
+ String payload = (String) topoConf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD);
+ if (Utils.isZkAuthenticationConfiguredStormServer(topoConf)) {
+ aclList = new ArrayList<>();
+ ACL acl1 = ZooDefs.Ids.CREATOR_ALL_ACL.get(0);
+ aclList.add(acl1);
+ ACL acl2 = new ACL(ZooDefs.Perms.READ, new Id("digest", DigestAuthenticationProvider.generateDigest(payload)));
+ aclList.add(acl2);
+ }
+ return aclList;
+ }
+
+ public static String supervisorPath(String id) {
+ return SUPERVISORS_SUBTREE + ZK_SEPERATOR + id;
+ }
+
+ public static String assignmentPath(String id) {
+ return ASSIGNMENTS_SUBTREE + ZK_SEPERATOR + id;
+ }
+
+ public static String blobstorePath(String key) {
+ return BLOBSTORE_SUBTREE + ZK_SEPERATOR + key;
+ }
+
+ public static String blobstoreMaxKeySequenceNumberPath(String key) {
+ return BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_SUBTREE + ZK_SEPERATOR + key;
+ }
+
+ public static String nimbusPath(String id) {
+ return NIMBUSES_SUBTREE + ZK_SEPERATOR + id;
+ }
+
+ public static String stormPath(String id) {
+ return STORMS_SUBTREE + ZK_SEPERATOR + id;
+ }
+
+ public static String workerbeatStormRoot(String stormId) {
+ return WORKERBEATS_SUBTREE + ZK_SEPERATOR + stormId;
+ }
+
+ public static String workerbeatPath(String stormId, String node, Long port) {
+ return workerbeatStormRoot(stormId) + ZK_SEPERATOR + node + "-" + port;
+ }
+
+ public static String backpressureStormRoot(String stormId) {
+ return BACKPRESSURE_SUBTREE + ZK_SEPERATOR + stormId;
+ }
+
+ public static String backpressurePath(String stormId, String node, Long port) {
+ return backpressureStormRoot(stormId) + ZK_SEPERATOR + node + "-" + port;
+ }
+
+ public static String errorStormRoot(String stormId) {
+ return ERRORS_SUBTREE + ZK_SEPERATOR + stormId;
+ }
+
+ public static String errorPath(String stormId, String componentId) {
+ try {
+ return errorStormRoot(stormId) + ZK_SEPERATOR + URLEncoder.encode(componentId, "UTF-8");
+ } catch (UnsupportedEncodingException e) {
+ throw Utils.wrapInRuntime(e);
+ }
+ }
+
+ public static String lastErrorPath(String stormId, String componentId) {
+ return errorPath(stormId, componentId) + "-last-error";
+ }
+
+ public static String credentialsPath(String stormId) {
+ return CREDENTIALS_SUBTREE + ZK_SEPERATOR + stormId;
+ }
+
+ public static String logConfigPath(String stormId) {
+ return LOGCONFIG_SUBTREE + ZK_SEPERATOR + stormId;
+ }
+
+ public static String profilerConfigPath(String stormId) {
+ return PROFILERCONFIG_SUBTREE + ZK_SEPERATOR + stormId;
+ }
+
+ public static String profilerConfigPath(String stormId, String host, Long port, ProfileAction requestType) {
+ return profilerConfigPath(stormId) + ZK_SEPERATOR + host + "_" + port + "_" + requestType;
+ }
+
+ public static <T> T maybeDeserialize(byte[] serialized, Class<T> clazz) {
+ if (serialized != null) {
+ return Utils.deserialize(serialized, clazz);
+ }
+ return null;
+ }
+
- // Ensures that we only return heartbeats for executors assigned to this worker
- public static Map<ExecutorInfo, ClusterWorkerHeartbeat> convertExecutorBeats(List<ExecutorInfo> executors, ClusterWorkerHeartbeat workerHeartbeat) {
- Map<ExecutorInfo, ClusterWorkerHeartbeat> executorWhb = new HashMap<>();
++ /**
++ * Ensures that we only return heartbeats for executors assigned to this worker
++ * @param executors
++ * @param workerHeartbeat
++ * @return
++ */
++ public static Map<ExecutorInfo, APersistentMap> convertExecutorBeats(List<ExecutorInfo> executors, ClusterWorkerHeartbeat workerHeartbeat) {
++ Map<ExecutorInfo, APersistentMap> executorWhb = new HashMap<>();
+ Map<ExecutorInfo, ExecutorStats> executorStatsMap = workerHeartbeat.get_executor_stats();
+ for (ExecutorInfo executor : executors) {
+ if (executorStatsMap.containsKey(executor)) {
- executorWhb.put(executor, workerHeartbeat);
++ APersistentMap executorBeat =
++ new PersistentArrayMap(new Object[] { RT.keyword(null, "time-secs"), workerHeartbeat.get_time_secs(), RT.keyword(null, "uptime"),
++ workerHeartbeat.get_uptime_secs(), RT.keyword(null, "stats"), workerHeartbeat.get_executor_stats().get(executor) });
++ executorWhb.put(executor, executorBeat);
+ }
+ }
+ return executorWhb;
+ }
+
+ public IStormClusterState mkStormClusterStateImpl(Object stateStorage, List<ACL> acls, ClusterStateContext context) throws Exception {
+ if (stateStorage instanceof IStateStorage) {
+ return new StormClusterStateImpl((IStateStorage) stateStorage, acls, context, false);
+ } else {
+ IStateStorage Storage = _instance.mkStateStorageImpl((APersistentMap) stateStorage, (APersistentMap) stateStorage, acls, context);
+ return new StormClusterStateImpl(Storage, acls, context, true);
+ }
+
+ }
+
- public IStateStorage mkStateStorageImpl(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context)
- throws Exception {
++ public IStateStorage mkStateStorageImpl(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
+ String className = null;
+ IStateStorage stateStorage = null;
+ if (config.get(Config.STORM_CLUSTER_STATE_STORE) != null) {
+ className = (String) config.get(Config.STORM_CLUSTER_STATE_STORE);
+ } else {
+ className = "org.apache.storm.cluster.ZKStateStorageFactory";
+ }
+ Class clazz = Class.forName(className);
+ StateStorageFactory storageFactory = (StateStorageFactory) clazz.newInstance();
+ stateStorage = storageFactory.mkStore(config, auth_conf, acls, context);
+ return stateStorage;
+ }
+
- public static IStateStorage mkStateStorage(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context)
- throws Exception {
++ public static IStateStorage mkStateStorage(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
+ return _instance.mkStateStorageImpl(config, auth_conf, acls, context);
+ }
+
+ public static IStormClusterState mkStormClusterState(Object StateStorage, List<ACL> acls, ClusterStateContext context) throws Exception {
+ return _instance.mkStormClusterStateImpl(StateStorage, acls, context);
+ }
+
+ // TO be remove
+ public static <K, V> HashMap<V, List<K>> reverseMap(Map<K, V> map) {
+ HashMap<V, List<K>> rtn = new HashMap<V, List<K>>();
+ if (map == null) {
+ return rtn;
+ }
+ for (Map.Entry<K, V> entry : map.entrySet()) {
+ K key = entry.getKey();
+ V val = entry.getValue();
+ List<K> list = rtn.get(val);
+ if (list == null) {
+ list = new ArrayList<K>();
+ rtn.put(entry.getValue(), list);
+ }
+ list.add(key);
+ }
+ return rtn;
+ }
++
++ public static String StringifyError(Throwable error) {
++ String errorString = null;
++ StringWriter result = null;
++ PrintWriter printWriter = null;
++ try {
++ result = new StringWriter();
++ printWriter = new PrintWriter(result);
++ error.printStackTrace(printWriter);
++ if (result != null) {
++ errorString = result.toString();
++ }
++ } finally {
++ try {
++ if (result != null)
++ result.close();
++ if (printWriter != null)
++ printWriter.close();
++ } catch (Exception e) {
++ }
++ }
++ return errorString;
++ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
index 59d1af7,0000000..01cf56a
mode 100644,000000..100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
@@@ -1,129 -1,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 org.apache.storm.cluster;
+
+import clojure.lang.APersistentMap;
+import clojure.lang.IFn;
+import org.apache.storm.generated.*;
+import org.apache.storm.nimbus.NimbusInfo;
+
+import java.security.NoSuchAlgorithmException;
+import java.util.List;
+import java.util.Map;
+
+public interface IStormClusterState {
+ public List<String> assignments(IFn callback);
+
+ public Assignment assignmentInfo(String stormId, IFn callback);
+
+ public APersistentMap assignmentInfoWithVersion(String stormId, IFn callback);
+
+ public Integer assignmentVersion(String stormId, IFn callback) throws Exception;
+
- // returns key information under /storm/blobstore/key
+ public List<String> blobstoreInfo(String blobKey);
+
- // returns list of nimbus summaries stored under /stormroot/nimbuses/<nimbus-ids> -> <data>
+ public List nimbuses();
+
- // adds the NimbusSummary to /stormroot/nimbuses/nimbus-id
+ public void addNimbusHost(String nimbusId, NimbusSummary nimbusSummary);
+
+ public List<String> activeStorms();
+
+ public StormBase stormBase(String stormId, IFn callback);
+
+ public ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, String node, Long port);
+
+ public List<ProfileRequest> getWorkerProfileRequests(String stormId, NodeInfo nodeInfo, boolean isThrift);
+
+ public List<ProfileRequest> getTopologyProfileRequests(String stormId, boolean isThrift);
+
+ public void setWorkerProfileRequest(String stormId, ProfileRequest profileRequest);
+
+ public void deleteTopologyProfileRequests(String stormId, ProfileRequest profileRequest);
+
- public Map<ExecutorInfo, ClusterWorkerHeartbeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort);
++ public Map<ExecutorInfo, APersistentMap> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort);
+
+ public List<String> supervisors(IFn callback);
+
+ public SupervisorInfo supervisorInfo(String supervisorId); // returns nil if doesn't exist
+
+ public void setupHeatbeats(String stormId);
+
+ public void teardownHeartbeats(String stormId);
+
+ public void teardownTopologyErrors(String stormId);
+
+ public List<String> heartbeatStorms();
+
+ public List<String> errorTopologies();
+
+ public void setTopologyLogConfig(String stormId, LogConfig logConfig);
+
+ public LogConfig topologyLogConfig(String stormId, IFn cb);
+
+ public void workerHeartbeat(String stormId, String node, Long port, ClusterWorkerHeartbeat info);
+
+ public void removeWorkerHeartbeat(String stormId, String node, Long port);
+
+ public void supervisorHeartbeat(String supervisorId, SupervisorInfo info);
+
+ public void workerBackpressure(String stormId, String node, Long port, boolean on);
+
+ public boolean topologyBackpressure(String stormId, IFn callback);
+
+ public void setupBackpressure(String stormId);
+
+ public void removeWorkerBackpressure(String stormId, String node, Long port);
+
+ public void activateStorm(String stormId, StormBase stormBase);
+
+ public void updateStorm(String stormId, StormBase newElems);
+
+ public void removeStormBase(String stormId);
+
+ public void setAssignment(String stormId, Assignment info);
+
- // sets up information related to key consisting of nimbus
- // host:port and version info of the blob
+ public void setupBlobstore(String key, NimbusInfo nimbusInfo, Integer versionInfo);
+
+ public List<String> activeKeys();
+
+ public List<String> blobstore(IFn callback);
+
+ public void removeStorm(String stormId);
+
+ public void removeBlobstoreKey(String blobKey);
+
+ public void removeKeyVersion(String blobKey);
+
- public void reportError(String stormId, String componentId, String node, Long port, String error);
++ public void reportError(String stormId, String componentId, String node, Long port, Throwable error);
+
+ public List<ErrorInfo> errors(String stormId, String componentId);
+
+ public ErrorInfo lastError(String stormId, String componentId);
+
+ public void setCredentials(String stormId, Credentials creds, Map topoConf) throws NoSuchAlgorithmException;
+
+ public Credentials credentials(String stormId, IFn callback);
+
+ public void disconnect();
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
index 1226c55,0000000..a9c4d89
mode 100644,000000..100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
@@@ -1,212 -1,0 +1,212 @@@
+/**
+ * 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 org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.storm.callback.ZKStateChangedCallback;
+import org.apache.storm.generated.*;
+import org.apache.storm.pacemaker.PacemakerClient;
+import org.apache.storm.utils.Utils;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+public class PaceMakerStateStorage implements IStateStorage {
+
+ private static Logger LOG = LoggerFactory.getLogger(PaceMakerStateStorage.class);
+
+ private PacemakerClient pacemakerClient;
+ private IStateStorage stateStorage;
+ private static final int maxRetries = 10;
+
+ public PaceMakerStateStorage(PacemakerClient pacemakerClient, IStateStorage stateStorage) throws Exception {
+ this.pacemakerClient = pacemakerClient;
+ this.stateStorage = stateStorage;
+ }
+
+ @Override
+ public String register(ZKStateChangedCallback callback) {
+ return stateStorage.register(callback);
+ }
+
+ @Override
+ public void unregister(String id) {
+ stateStorage.unregister(id);
+ }
+
+ @Override
+ public String create_sequential(String path, byte[] data, List<ACL> acls) {
+ return stateStorage.create_sequential(path, data, acls);
+ }
+
+ @Override
+ public void mkdirs(String path, List<ACL> acls) {
+ stateStorage.mkdirs(path, acls);
+ }
+
+ @Override
+ public void delete_node(String path) {
+ stateStorage.delete_node(path);
+ }
+
+ @Override
+ public void set_ephemeral_node(String path, byte[] data, List<ACL> acls) {
+ stateStorage.set_ephemeral_node(path, data, acls);
+ }
+
+ @Override
+ public Integer get_version(String path, boolean watch) throws Exception {
+ return stateStorage.get_version(path, watch);
+ }
+
+ @Override
+ public boolean node_exists(String path, boolean watch) {
+ return stateStorage.node_exists(path, watch);
+ }
+
+ @Override
+ public List<String> get_children(String path, boolean watch) {
+ return stateStorage.get_children(path, watch);
+ }
+
+ @Override
+ public void close() {
+ stateStorage.close();
+ pacemakerClient.close();
+ }
+
+ @Override
+ public void set_data(String path, byte[] data, List<ACL> acls) {
+ stateStorage.set_data(path, data, acls);
+ }
+
+ @Override
+ public byte[] get_data(String path, boolean watch) {
+ return stateStorage.get_data(path, watch);
+ }
+
+ @Override
+ public APersistentMap get_data_with_version(String path, boolean watch) {
+ return stateStorage.get_data_with_version(path, watch);
+ }
+
+ @Override
+ public void set_worker_hb(String path, byte[] data, List<ACL> acls) {
+ int retry = maxRetries;
+ while (true) {
+ try {
+ HBPulse hbPulse = new HBPulse();
+ hbPulse.set_id(path);
+ hbPulse.set_details(data);
+ HBMessage message = new HBMessage(HBServerMessageType.SEND_PULSE, HBMessageData.pulse(hbPulse));
+ HBMessage response = pacemakerClient.send(message);
+ if (response.get_type() != HBServerMessageType.SEND_PULSE_RESPONSE) {
+ throw new HBExecutionException("Invalid Response Type");
+ }
+ LOG.debug("Successful set_worker_hb");
+ break;
+ } catch (Exception e) {
+ if (retry <= 0) {
+ throw Utils.wrapInRuntime(e);
+ }
+ LOG.error("{} Failed to set_worker_hb. Will make {} more attempts.", e.getMessage(), retry--);
+ }
+ }
+ }
+
+ @Override
+ public byte[] get_worker_hb(String path, boolean watch) {
+ int retry = maxRetries;
+ while (true) {
+ try {
+ HBMessage message = new HBMessage(HBServerMessageType.GET_PULSE, HBMessageData.path(path));
+ HBMessage response = pacemakerClient.send(message);
+ if (response.get_type() != HBServerMessageType.GET_PULSE_RESPONSE) {
+ throw new HBExecutionException("Invalid Response Type");
+ }
+ LOG.debug("Successful get_worker_hb");
+ return response.get_data().get_pulse().get_details();
+ } catch (Exception e) {
+ if (retry <= 0) {
+ throw Utils.wrapInRuntime(e);
+ }
+ LOG.error("{} Failed to get_worker_hb. Will make {} more attempts.", e.getMessage(), retry--);
+ }
+ }
+ }
+
+ @Override
+ public List<String> get_worker_hb_children(String path, boolean watch) {
+ int retry = maxRetries;
+ while (true) {
+ try {
- HBMessage message = new HBMessage(HBServerMessageType.GET_PULSE, HBMessageData.path(path));
++ HBMessage message = new HBMessage(HBServerMessageType.GET_ALL_NODES_FOR_PATH, HBMessageData.path(path));
+ HBMessage response = pacemakerClient.send(message);
+ if (response.get_type() != HBServerMessageType.GET_ALL_NODES_FOR_PATH_RESPONSE) {
+ throw new HBExecutionException("Invalid Response Type");
+ }
+ LOG.debug("Successful get_worker_hb");
+ return response.get_data().get_nodes().get_pulseIds();
+ } catch (Exception e) {
+ if (retry <= 0) {
+ throw Utils.wrapInRuntime(e);
+ }
+ LOG.error("{} Failed to get_worker_hb_children. Will make {} more attempts.", e.getMessage(), retry--);
+ }
+ }
+ }
+
+ @Override
+ public void delete_worker_hb(String path) {
+ int retry = maxRetries;
+ while (true) {
+ try {
- HBMessage message = new HBMessage(HBServerMessageType.GET_PULSE, HBMessageData.path(path));
++ HBMessage message = new HBMessage(HBServerMessageType.DELETE_PATH, HBMessageData.path(path));
+ HBMessage response = pacemakerClient.send(message);
+ if (response.get_type() != HBServerMessageType.DELETE_PATH_RESPONSE) {
+ throw new HBExecutionException("Invalid Response Type");
+ }
+ LOG.debug("Successful get_worker_hb");
+ break;
+ } catch (Exception e) {
+ if (retry <= 0) {
+ throw Utils.wrapInRuntime(e);
+ }
+ LOG.error("{} Failed to delete_worker_hb. Will make {} more attempts.", e.getMessage(), retry--);
+ }
+ }
+ }
+
+ @Override
+ public void add_listener(ConnectionStateListener listener) {
+ stateStorage.add_listener(listener);
+ }
+
+ @Override
+ public void sync_path(String path) {
+ stateStorage.sync_path(path);
+ }
+
+ @Override
+ public void delete_node_blobstore(String path, String nimbusHostPortInfo) {
+ stateStorage.delete_node_blobstore(path, nimbusHostPortInfo);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
index c2477d6,0000000..110da41
mode 100644,000000..100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
@@@ -1,28 -1,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.
+ */
+package org.apache.storm.cluster;
+
+import clojure.lang.APersistentMap;
+import java.util.List;
+import org.apache.zookeeper.data.ACL;
+
+public interface StateStorageFactory {
-
++
+ IStateStorage mkStore(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context);
+
+}
[15/27] storm git commit: remove some useless code
Posted by bo...@apache.org.
remove some useless code
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/419be35d
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/419be35d
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/419be35d
Branch: refs/heads/master
Commit: 419be35d5ed0cbbc9abdf89058c1b484ecbf8e6d
Parents: 924f9a2 9a8962d
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Tue Feb 16 11:17:22 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Tue Feb 16 11:17:22 2016 +0800
----------------------------------------------------------------------
storm-core/src/clj/org/apache/storm/daemon/supervisor.clj | 3 ++-
storm-core/test/clj/org/apache/storm/nimbus_test.clj | 2 ++
2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
[22/27] storm git commit: Merge branch 'master' into ClusterUtils
Posted by bo...@apache.org.
Merge branch 'master' into ClusterUtils
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/581cd298
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/581cd298
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/581cd298
Branch: refs/heads/master
Commit: 581cd29876fab988477ac661174cad5ffbeced1e
Parents: 3717f39 58050a5
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Tue Feb 23 19:54:31 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Tue Feb 23 19:56:06 2016 +0800
----------------------------------------------------------------------
CHANGELOG.md | 2 +
.../src/clj/org/apache/storm/daemon/nimbus.clj | 2 +-
.../clj/org/apache/storm/daemon/supervisor.clj | 8 +-
.../src/clj/org/apache/storm/daemon/worker.clj | 3 +-
.../clj/org/apache/storm/process_simulator.clj | 49 ------------
storm-core/src/clj/org/apache/storm/testing.clj | 9 ++-
storm-core/src/clj/org/apache/storm/ui/core.clj | 4 +-
.../jvm/org/apache/storm/ProcessSimulator.java | 82 ++++++++++++++++++++
8 files changed, 98 insertions(+), 61 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/581cd298/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index 7d28075,28a6fb8..e43bab9
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@@ -1212,10 -1212,10 +1212,10 @@@
(.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)))))
+ (.setupBlobstore 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]
- (->> (apply clojurify-error (.errors storm-cluster-state storm-id component-id))
- (->> (.errors storm-cluster-state storm-id component-id)
++ (->> (map clojurify-error (.errors storm-cluster-state storm-id component-id))
(map #(doto (ErrorInfo. (:error %) (:time-secs %))
(.set_host (:host %))
(.set_port (:port %))))))
http://git-wip-us.apache.org/repos/asf/storm/blob/581cd298/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
index a6f3a62,a34d461..0cee414
--- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
@@@ -35,7 -35,7 +35,7 @@@
(:use [org.apache.storm.daemon common])
(:import [org.apache.storm.command HealthCheck])
(:require [org.apache.storm.daemon [worker :as worker]]
- [org.apache.storm [process-simulator :as psim]]
- [org.apache.storm [cluster :as cluster]]
++
[clojure.set :as set])
(:import [org.apache.thrift.transport TTransportException])
(:import [org.apache.zookeeper data.ACL ZooDefs$Ids ZooDefs$Perms])
http://git-wip-us.apache.org/repos/asf/storm/blob/581cd298/storm-core/src/clj/org/apache/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/worker.clj
index a75dc35,db4a61a..af88f6a
--- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
@@@ -19,8 -19,9 +19,9 @@@
(:require [clj-time.core :as time])
(:require [clj-time.coerce :as coerce])
(:require [org.apache.storm.daemon [executor :as executor]])
- (:require [org.apache.storm [cluster :as cluster]])
+
(:require [clojure.set :as set])
+ (:import [java.io File])
(:import [java.util.concurrent Executors]
[org.apache.storm.hooks IWorkerHook BaseWorkerHook]
[uk.org.lidalia.sysoutslf4j.context SysOutOverSLF4J])
http://git-wip-us.apache.org/repos/asf/storm/blob/581cd298/storm-core/src/clj/org/apache/storm/testing.clj
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/581cd298/storm-core/src/clj/org/apache/storm/ui/core.clj
----------------------------------------------------------------------
[08/27] storm git commit: update class hierarchy about cluster
Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
new file mode 100644
index 0000000..cd2bc4a
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
@@ -0,0 +1,664 @@
+/**
+ * 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.*;
+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.ZKStateChangedCallback;
+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.Watcher;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.security.NoSuchAlgorithmException;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+
+public class StormClusterStateImpl implements StormClusterState {
+
+ private static Logger LOG = LoggerFactory.getLogger(StormClusterStateImpl.class);
+
+ private StateStorage stateStorage;
+
+ 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 StormClusterStateImpl(StateStorage StateStorage, List<ACL> acls, ClusterStateContext context, boolean solo) throws Exception {
+
+ this.stateStorage = StateStorage;
+ this.solo = solo;
+
+ 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.stateStorage.register(new ZKStateChangedCallback() {
+
+ public void changed(Watcher.Event.EventType type, String path) {
+ 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(ClusterUtils.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(ClusterUtils.SUPERVISORS_ROOT)) {
+ issueCallback(supervisorsCallback);
+ } else if (root.equals(ClusterUtils.BLOBSTORE_ROOT)) {
+ issueCallback(blobstoreCallback);
+ } else if (root.equals(ClusterUtils.STORMS_ROOT) && size > 1) {
+ issueMapCallback(stormBaseCallback, toks.get(1));
+ } else if (root.equals(ClusterUtils.CREDENTIALS_ROOT) && size > 1) {
+ issueMapCallback(credentialsCallback, toks.get(1));
+ } else if (root.equals(ClusterUtils.LOGCONFIG_ROOT) && size > 1) {
+ issueMapCallback(logConfigCallback, toks.get(1));
+ } else if (root.equals(ClusterUtils.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;
+ }
+
+ });
+
+ String[] pathlist = { ClusterUtils.ASSIGNMENTS_SUBTREE, ClusterUtils.STORMS_SUBTREE, ClusterUtils.SUPERVISORS_SUBTREE, ClusterUtils.WORKERBEATS_SUBTREE,
+ ClusterUtils.ERRORS_SUBTREE, ClusterUtils.BLOBSTORE_SUBTREE, ClusterUtils.NIMBUSES_SUBTREE, ClusterUtils.LOGCONFIG_SUBTREE };
+ for (String path : pathlist) {
+ this.stateStorage.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 stateStorage.get_children(ClusterUtils.ASSIGNMENTS_SUBTREE, callback != null);
+ }
+
+ @Override
+ public Assignment assignmentInfo(String stormId, IFn callback) {
+ if (callback != null) {
+ assignmentInfoCallback.put(stormId, callback);
+ }
+ byte[] serialized = stateStorage.get_data(ClusterUtils.assignmentPath(stormId), callback != null);
+ return ClusterUtils.maybeDeserialize(serialized, Assignment.class);
+ }
+
+ @Override
+ public APersistentMap assignmentInfoWithVersion(String stormId, IFn callback) {
+ if (callback != null) {
+ assignmentInfoWithVersionCallback.put(stormId, callback);
+ }
+ Assignment assignment = null;
+ Integer version = 0;
+ APersistentMap aPersistentMap = stateStorage.get_data_with_version(ClusterUtils.assignmentPath(stormId), callback != null);
+ if (aPersistentMap != null) {
+ assignment = ClusterUtils.maybeDeserialize((byte[]) aPersistentMap.get(RT.keyword(null, "data")), Assignment.class);
+ version = (Integer) aPersistentMap.get(RT.keyword(null, "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 stateStorage.get_version(ClusterUtils.assignmentPath(stormId), callback != null);
+ }
+
+ // blobstore state
+ @Override
+ public List<String> blobstoreInfo(String blobKey) {
+ String path = ClusterUtils.blobstorePath(blobKey);
+ stateStorage.sync_path(path);
+ return stateStorage.get_children(path, false);
+ }
+
+ @Override
+ public List nimbuses() {
+ List<NimbusSummary> nimbusSummaries = new ArrayList<>();
+ List<String> nimbusIds = stateStorage.get_children(ClusterUtils.NIMBUSES_SUBTREE, false);
+ for (String nimbusId : nimbusIds) {
+ byte[] serialized = stateStorage.get_data(ClusterUtils.nimbusPath(nimbusId), false);
+ NimbusSummary nimbusSummary = ClusterUtils.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.
+ stateStorage.delete_node(ClusterUtils.nimbusPath(nimbusId));
+ stateStorage.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");
+ stateStorage.set_ephemeral_node(ClusterUtils.nimbusPath(nimbusId), Utils.serialize(nimbusSummary), acls);
+ }
+
+ }
+ });
+
+ stateStorage.set_ephemeral_node(ClusterUtils.nimbusPath(nimbusId), Utils.serialize(nimbusSummary), acls);
+ }
+
+ @Override
+ public List<String> activeStorms() {
+ return stateStorage.get_children(ClusterUtils.STORMS_SUBTREE, false);
+ }
+
+ @Override
+ public StormBase stormBase(String stormId, IFn callback) {
+ if (callback != null) {
+ stormBaseCallback.put(stormId, callback);
+ }
+ return ClusterUtils.maybeDeserialize(stateStorage.get_data(ClusterUtils.stormPath(stormId), callback != null), StormBase.class);
+ }
+
+ @Override
+ public ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, String node, Long port) {
+ byte[] bytes = stateStorage.get_worker_hb(ClusterUtils.workerbeatPath(stormId, node, port), false);
+ return ClusterUtils.maybeDeserialize(bytes, ClusterWorkerHeartbeat.class);
+
+ }
+
+ @Override
+ public List<ProfileRequest> getWorkerProfileRequests(String stormId, NodeInfo nodeInfo, boolean isThrift) {
+ List<ProfileRequest> requests = new ArrayList<>();
+ List<ProfileRequest> profileRequests = getTopologyProfileRequests(stormId, isThrift);
+ for (ProfileRequest profileRequest : profileRequests) {
+ NodeInfo nodeInfo1 = profileRequest.get_nodeInfo();
+ if (nodeInfo1.equals(nodeInfo))
+ requests.add(profileRequest);
+ }
+ return requests;
+ }
+
+ @Override
+ public List<ProfileRequest> getTopologyProfileRequests(String stormId, boolean isThrift) {
+ List<ProfileRequest> profileRequests = new ArrayList<>();
+ String path = ClusterUtils.profilerConfigPath(stormId);
+ if (stateStorage.node_exists(path, false)) {
+ List<String> strs = stateStorage.get_children(path, false);
+ for (String str : strs) {
+ String childPath = path + ClusterUtils.ZK_SEPERATOR + str;
+ byte[] raw = stateStorage.get_data(childPath, false);
+ ProfileRequest request = ClusterUtils.maybeDeserialize(raw, ProfileRequest.class);
+ if (request != null)
+ profileRequests.add(request);
+ }
+ }
+ return profileRequests;
+ }
+
+ @Override
+ public void setWorkerProfileRequest(String stormId, ProfileRequest profileRequest) {
+ ProfileAction profileAction = profileRequest.get_action();
+ String host = profileRequest.get_nodeInfo().get_node();
+ Long port = profileRequest.get_nodeInfo().get_port_iterator().next();
+ String path = ClusterUtils.profilerConfigPath(stormId, host, port, profileAction);
+ stateStorage.set_data(path, Utils.serialize(profileRequest), acls);
+ }
+
+ @Override
+ public void deleteTopologyProfileRequests(String stormId, ProfileRequest profileRequest) {
+ ProfileAction profileAction = profileRequest.get_action();
+ String host = profileRequest.get_nodeInfo().get_node();
+ Long port = profileRequest.get_nodeInfo().get_port_iterator().next();
+ String path = ClusterUtils.profilerConfigPath(stormId, host, port, profileAction);
+ stateStorage.delete_node(path);
+ }
+
+ // need to take executor->node+port in explicitly so that we don't run into a situation where a
+ // long dead worker with a skewed clock overrides all the timestamps. By only checking heartbeats
+ // with an assigned node+port, and only reading executors from that heartbeat that are actually assigned,
+ // we avoid situations like that
+ @Override
+ public Map<ExecutorInfo, ClusterWorkerHeartbeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort) {
+ Map<ExecutorInfo, ClusterWorkerHeartbeat> executorWhbs = new HashMap<>();
+
+ Map<NodeInfo, List<List<Long>>> nodePortExecutors = ClusterUtils.reverseMap(executorNodePort);
+
+ for (Map.Entry<NodeInfo, List<List<Long>>> entry : nodePortExecutors.entrySet()) {
+
+ String node = entry.getKey().get_node();
+ Long port = entry.getKey().get_port_iterator().next();
+ ClusterWorkerHeartbeat whb = getWorkerHeartbeat(stormId, node, port);
+ List<ExecutorInfo> executorInfoList = new ArrayList<>();
+ for (List<Long> list : entry.getValue()) {
+ executorInfoList.add(new ExecutorInfo(list.get(0).intValue(), list.get(list.size() - 1).intValue()));
+ }
+ if (whb != null)
+ executorWhbs.putAll(ClusterUtils.convertExecutorBeats(executorInfoList, whb));
+ }
+ return executorWhbs;
+ }
+
+ @Override
+ public List<String> supervisors(IFn callback) {
+ if (callback != null) {
+ supervisorsCallback.set(callback);
+ }
+ return stateStorage.get_children(ClusterUtils.SUPERVISORS_SUBTREE, callback != null);
+ }
+
+ @Override
+ public SupervisorInfo supervisorInfo(String supervisorId) {
+ String path = ClusterUtils.supervisorPath(supervisorId);
+ return ClusterUtils.maybeDeserialize(stateStorage.get_data(path, false), SupervisorInfo.class);
+ }
+
+ @Override
+ public void setupHeatbeats(String stormId) {
+ stateStorage.mkdirs(ClusterUtils.workerbeatStormRoot(stormId), acls);
+ }
+
+ @Override
+ public void teardownHeartbeats(String stormId) {
+ try {
+ stateStorage.delete_worker_hb(ClusterUtils.workerbeatStormRoot(stormId));
+ } catch (Exception e) {
+ if (Zookeeper.exceptionCause(KeeperException.class, e)) {
+ // do nothing
+ LOG.warn("Could not teardown heartbeats for {}.", stormId);
+ } else {
+ throw e;
+ }
+ }
+ }
+
+ @Override
+ public void teardownTopologyErrors(String stormId) {
+ try {
+ stateStorage.delete_node(ClusterUtils.errorStormRoot(stormId));
+ } catch (Exception e) {
+ if (Zookeeper.exceptionCause(KeeperException.class, e)) {
+ // do nothing
+ LOG.warn("Could not teardown errors for {}.", stormId);
+ } else {
+ throw e;
+ }
+ }
+ }
+
+ @Override
+ public List<String> heartbeatStorms() {
+ return stateStorage.get_worker_hb_children(ClusterUtils.WORKERBEATS_SUBTREE, false);
+ }
+
+ @Override
+ public List<String> errorTopologies() {
+ return stateStorage.get_children(ClusterUtils.ERRORS_SUBTREE, false);
+ }
+
+ @Override
+ public void setTopologyLogConfig(String stormId, LogConfig logConfig) {
+ stateStorage.set_data(ClusterUtils.logConfigPath(stormId), Utils.serialize(logConfig), acls);
+ }
+
+ @Override
+ public LogConfig topologyLogConfig(String stormId, IFn cb) {
+ String path = ClusterUtils.logConfigPath(stormId);
+ return ClusterUtils.maybeDeserialize(stateStorage.get_data(path, cb != null), LogConfig.class);
+ }
+
+ @Override
+ public void workerHeartbeat(String stormId, String node, Long port, ClusterWorkerHeartbeat info) {
+ if (info != null) {
+ String path = ClusterUtils.workerbeatPath(stormId, node, port);
+ stateStorage.set_worker_hb(path, Utils.serialize(info), acls);
+ }
+ }
+
+ @Override
+ public void removeWorkerHeartbeat(String stormId, String node, Long port) {
+ String path = ClusterUtils.workerbeatPath(stormId, node, port);
+ stateStorage.delete_worker_hb(path);
+ }
+
+ @Override
+ public void supervisorHeartbeat(String supervisorId, SupervisorInfo info) {
+ String path = ClusterUtils.supervisorPath(supervisorId);
+ stateStorage.set_ephemeral_node(path, Utils.serialize(info), acls);
+ }
+
+ // if znode exists and to be not on?, delete; if exists and on?, do nothing;
+ // if not exists and to be on?, create; if not exists and not on?, do nothing;
+ @Override
+ public void workerBackpressure(String stormId, String node, Long port, boolean on) {
+ String path = ClusterUtils.backpressurePath(stormId, node, port);
+ boolean existed = stateStorage.node_exists(path, false);
+ if (existed) {
+ if (on == false)
+ stateStorage.delete_node(path);
+
+ } else {
+ if (on == true) {
+ stateStorage.set_ephemeral_node(path, null, acls);
+ }
+ }
+ }
+
+ // if the backpresure/storm-id dir is empty, this topology has throttle-on, otherwise not.
+ @Override
+ public boolean topologyBackpressure(String stormId, IFn callback) {
+ if (callback != null) {
+ backPressureCallback.put(stormId, callback);
+ }
+ String path = ClusterUtils.backpressureStormRoot(stormId);
+ List<String> childrens = stateStorage.get_children(path, callback != null);
+ return childrens.size() > 0;
+
+ }
+
+ @Override
+ public void setupBackpressure(String stormId) {
+ stateStorage.mkdirs(ClusterUtils.backpressureStormRoot(stormId), acls);
+ }
+
+ @Override
+ public void removeWorkerBackpressure(String stormId, String node, Long port) {
+ stateStorage.delete_node(ClusterUtils.backpressurePath(stormId, node, port));
+ }
+
+ @Override
+ public void activateStorm(String stormId, StormBase stormBase) {
+ String path = ClusterUtils.stormPath(stormId);
+ stateStorage.set_data(path, Utils.serialize(stormBase), acls);
+ }
+
+ // To update this function due to APersistentMap/APersistentSet is clojure's structure
+ @Override
+ public void updateStorm(String stormId, StormBase newElems) {
+
+ StormBase stormBase = stormBase(stormId, null);
+ if (stormBase.get_component_executors() != null) {
+
+ Map<String, Integer> newComponentExecutors = new HashMap<>();
+ Map<String, Integer> componentExecutors = newElems.get_component_executors();
+ // componentExecutors maybe be APersistentMap, which don't support "put"
+ for (Map.Entry<String, Integer> entry : componentExecutors.entrySet()) {
+ newComponentExecutors.put(entry.getKey(), entry.getValue());
+ }
+ for (Map.Entry<String, Integer> entry : stormBase.get_component_executors().entrySet()) {
+ if (!componentExecutors.containsKey(entry.getKey())) {
+ newComponentExecutors.put(entry.getKey(), entry.getValue());
+ }
+ }
+ if (newComponentExecutors.size() > 0)
+ newElems.set_component_executors(newComponentExecutors);
+ }
+
+ Map<String, DebugOptions> ComponentDebug = new HashMap<>();
+ Map<String, DebugOptions> oldComponentDebug = stormBase.get_component_debug();
+
+ Map<String, DebugOptions> newComponentDebug = newElems.get_component_debug();
+ /// oldComponentDebug.keySet()/ newComponentDebug.keySet() maybe be APersistentSet, which don't support addAll
+ Set<String> debugOptionsKeys = new HashSet<>();
+ debugOptionsKeys.addAll(oldComponentDebug.keySet());
+ debugOptionsKeys.addAll(newComponentDebug.keySet());
+ for (String key : debugOptionsKeys) {
+ boolean enable = false;
+ double samplingpct = 0;
+ if (oldComponentDebug.containsKey(key)) {
+ enable = oldComponentDebug.get(key).is_enable();
+ samplingpct = oldComponentDebug.get(key).get_samplingpct();
+ }
+ if (newComponentDebug.containsKey(key)) {
+ enable = newComponentDebug.get(key).is_enable();
+ samplingpct += newComponentDebug.get(key).get_samplingpct();
+ }
+ DebugOptions debugOptions = new DebugOptions();
+ debugOptions.set_enable(enable);
+ debugOptions.set_samplingpct(samplingpct);
+ ComponentDebug.put(key, debugOptions);
+ }
+ if (ComponentDebug.size() > 0) {
+ newElems.set_component_debug(ComponentDebug);
+ }
+
+ if (StringUtils.isBlank(newElems.get_name())) {
+ newElems.set_name(stormBase.get_name());
+ }
+ if (newElems.get_status() == null) {
+ newElems.set_status(stormBase.get_status());
+ }
+ if (newElems.get_num_workers() == 0) {
+ newElems.set_num_workers(stormBase.get_num_workers());
+ }
+ if (newElems.get_launch_time_secs() == 0) {
+ newElems.set_launch_time_secs(stormBase.get_launch_time_secs());
+ }
+ if (StringUtils.isBlank(newElems.get_owner())) {
+ newElems.set_owner(stormBase.get_owner());
+ }
+ if (newElems.get_topology_action_options() == null) {
+ newElems.set_topology_action_options(stormBase.get_topology_action_options());
+ }
+ if (newElems.get_status() == null) {
+ newElems.set_status(stormBase.get_status());
+ }
+ stateStorage.set_data(ClusterUtils.stormPath(stormId), Utils.serialize(newElems), acls);
+ }
+
+ @Override
+ public void removeStormBase(String stormId) {
+ stateStorage.delete_node(ClusterUtils.stormPath(stormId));
+ }
+
+ @Override
+ public void setAssignment(String stormId, Assignment info) {
+ stateStorage.set_data(ClusterUtils.assignmentPath(stormId), Utils.serialize(info), acls);
+ }
+
+ @Override
+ public void setupBlobstore(String key, NimbusInfo nimbusInfo, Integer versionInfo) {
+ String path = ClusterUtils.blobstorePath(key) + ClusterUtils.ZK_SEPERATOR + nimbusInfo.toHostPortString() + "-" + versionInfo;
+ LOG.info("set-path: {}", path);
+ stateStorage.mkdirs(ClusterUtils.blobstorePath(key), acls);
+ stateStorage.delete_node_blobstore(ClusterUtils.blobstorePath(key), nimbusInfo.toHostPortString());
+ stateStorage.set_ephemeral_node(path, null, acls);
+ }
+
+ @Override
+ public List<String> activeKeys() {
+ return stateStorage.get_children(ClusterUtils.BLOBSTORE_SUBTREE, false);
+ }
+
+ // blobstore state
+ @Override
+ public List<String> blobstore(IFn callback) {
+ if (callback != null) {
+ blobstoreCallback.set(callback);
+ }
+ stateStorage.sync_path(ClusterUtils.BLOBSTORE_SUBTREE);
+ return stateStorage.get_children(ClusterUtils.BLOBSTORE_SUBTREE, callback != null);
+
+ }
+
+ @Override
+ public void removeStorm(String stormId) {
+ stateStorage.delete_node(ClusterUtils.assignmentPath(stormId));
+ stateStorage.delete_node(ClusterUtils.credentialsPath(stormId));
+ stateStorage.delete_node(ClusterUtils.logConfigPath(stormId));
+ stateStorage.delete_node(ClusterUtils.profilerConfigPath(stormId));
+ removeStormBase(stormId);
+ }
+
+ @Override
+ public void removeBlobstoreKey(String blobKey) {
+ LOG.debug("remove key {}", blobKey);
+ stateStorage.delete_node(ClusterUtils.blobstorePath(blobKey));
+ }
+
+ @Override
+ public void removeKeyVersion(String blobKey) {
+ stateStorage.delete_node(ClusterUtils.blobstoreMaxKeySequenceNumberPath(blobKey));
+ }
+
+ @Override
+ public void reportError(String stormId, String componentId, String node, Long port, String error) {
+
+ String path = ClusterUtils.errorPath(stormId, componentId);
+ String lastErrorPath = ClusterUtils.lastErrorPath(stormId, componentId);
+ ErrorInfo errorInfo = new ErrorInfo(error, Time.currentTimeSecs());
+ errorInfo.set_host(node);
+ errorInfo.set_port(port.intValue());
+ byte[] serData = Utils.serialize(errorInfo);
+ stateStorage.mkdirs(path, acls);
+ stateStorage.create_sequential(path + ClusterUtils.ZK_SEPERATOR + "e", serData, acls);
+ stateStorage.set_data(lastErrorPath, serData, acls);
+ List<String> childrens = stateStorage.get_children(path, false);
+
+ Collections.sort(childrens, new Comparator<String>() {
+ public int compare(String arg0, String arg1) {
+ return Long.compare(Long.parseLong(arg0.substring(1)), Long.parseLong(arg1.substring(1)));
+ }
+ });
+
+ while (childrens.size() > 10) {
+ stateStorage.delete_node(path + ClusterUtils.ZK_SEPERATOR + childrens.remove(0));
+ }
+ }
+
+ @Override
+ public List<ErrorInfo> errors(String stormId, String componentId) {
+ List<ErrorInfo> errorInfos = new ArrayList<>();
+ try {
+ String path = ClusterUtils.errorPath(stormId, componentId);
+ if (stateStorage.node_exists(path, false)) {
+ List<String> childrens = stateStorage.get_children(path, false);
+ for (String child : childrens) {
+ String childPath = path + ClusterUtils.ZK_SEPERATOR + child;
+ ErrorInfo errorInfo = ClusterUtils.maybeDeserialize(stateStorage.get_data(childPath, false), ErrorInfo.class);
+ if (errorInfo != null)
+ errorInfos.add(errorInfo);
+ }
+ }
+ Collections.sort(errorInfos, new Comparator<ErrorInfo>() {
+ public int compare(ErrorInfo arg0, ErrorInfo arg1) {
+ return -Integer.compare(arg0.get_error_time_secs(), arg1.get_error_time_secs());
+ }
+ });
+ } catch (Exception e) {
+ throw Utils.wrapInRuntime(e);
+ }
+
+ return errorInfos;
+ }
+
+ @Override
+ public ErrorInfo lastError(String stormId, String componentId) {
+
+ String path = ClusterUtils.lastErrorPath(stormId, componentId);
+ if (stateStorage.node_exists(path, false)) {
+ ErrorInfo errorInfo = ClusterUtils.maybeDeserialize(stateStorage.get_data(path, false), ErrorInfo.class);
+ return errorInfo;
+ }
+
+ return null;
+ }
+
+ @Override
+ public void setCredentials(String stormId, Credentials creds, Map topoConf) throws NoSuchAlgorithmException {
+ List<ACL> aclList = ClusterUtils.mkTopoOnlyAcls(topoConf);
+ String path = ClusterUtils.credentialsPath(stormId);
+ stateStorage.set_data(path, Utils.serialize(creds), aclList);
+
+ }
+
+ @Override
+ public Credentials credentials(String stormId, IFn callback) {
+ if (callback != null) {
+ credentialsCallback.put(stormId, callback);
+ }
+ String path = ClusterUtils.credentialsPath(stormId);
+ return ClusterUtils.maybeDeserialize(stateStorage.get_data(path, callback != null), Credentials.class);
+
+ }
+
+ @Override
+ public void disconnect() {
+ stateStorage.unregister(stateId);
+ if (solo)
+ stateStorage.close();
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
deleted file mode 100644
index 3a4205b..0000000
--- a/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
+++ /dev/null
@@ -1,683 +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.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) {
- return Cluster.maybeDeserialize(bytes, ClusterWorkerHeartbeat.class);
- }
- return null;
- }
-
- @Override
- public List<ProfileRequest> getWorkerProfileRequests(String stormId, NodeInfo nodeInfo, boolean isThrift) {
- List<ProfileRequest> requests = new ArrayList<>();
- List<ProfileRequest> profileRequests = getTopologyProfileRequests(stormId, isThrift);
- for (ProfileRequest profileRequest : profileRequests) {
- NodeInfo nodeInfo1 = profileRequest.get_nodeInfo();
- if (nodeInfo1.equals(nodeInfo))
- requests.add(profileRequest);
- }
- return requests;
- }
-
- @Override
- public List<ProfileRequest> getTopologyProfileRequests(String stormId, boolean isThrift) {
- List<ProfileRequest> profileRequests = new ArrayList<>();
- String path = Cluster.profilerConfigPath(stormId);
- if (clusterState.node_exists(path, false)) {
- List<String> strs = clusterState.get_children(path, false);
- for (String str : strs) {
- String childPath = path + Cluster.ZK_SEPERATOR + str;
- byte[] raw = clusterState.get_data(childPath, false);
- ProfileRequest request = Cluster.maybeDeserialize(raw, ProfileRequest.class);
- if (request != null)
- profileRequests.add(request);
- }
- }
- return profileRequests;
- }
-
- @Override
- public void setWorkerProfileRequest(String stormId, ProfileRequest profileRequest) {
- ProfileAction profileAction = profileRequest.get_action();
- String host = profileRequest.get_nodeInfo().get_node();
- Long port = profileRequest.get_nodeInfo().get_port_iterator().next();
- String path = Cluster.profilerConfigPath(stormId, host, port, profileAction);
- clusterState.set_data(path, Utils.serialize(profileRequest), acls);
- }
-
- @Override
- public void deleteTopologyProfileRequests(String stormId, ProfileRequest profileRequest) {
- ProfileAction profileAction = profileRequest.get_action();
- String host = profileRequest.get_nodeInfo().get_node();
- Long port = profileRequest.get_nodeInfo().get_port_iterator().next();
- String path = Cluster.profilerConfigPath(stormId, host, port, profileAction);
- clusterState.delete_node(path);
- }
-
- // need to take executor->node+port in explicitly so that we don't run into a situation where a
- // long dead worker with a skewed clock overrides all the timestamps. By only checking heartbeats
- // with an assigned node+port, and only reading executors from that heartbeat that are actually assigned,
- // we avoid situations like that
- @Override
- public Map<ExecutorInfo, ClusterWorkerHeartbeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort) {
- Map<ExecutorInfo, ClusterWorkerHeartbeat> executorWhbs = new HashMap<>();
-
- LOG.info(executorNodePort.toString());
- Map<NodeInfo, List<List<Long>>> nodePortExecutors = Cluster.reverseMap(executorNodePort);
- LOG.info(nodePortExecutors.toString());
-
- for (Map.Entry<NodeInfo, List<List<Long>>> entry : nodePortExecutors.entrySet()) {
-
- String node = entry.getKey().get_node();
- Long port = entry.getKey().get_port_iterator().next();
- ClusterWorkerHeartbeat whb = getWorkerHeartbeat(stormId, node, port);
- List<ExecutorInfo> executorInfoList = new ArrayList<>();
- for (List<Long> list : entry.getValue()) {
- executorInfoList.add(new ExecutorInfo(list.get(0).intValue(), list.get(list.size() - 1).intValue()));
- }
- executorWhbs.putAll(Cluster.convertExecutorBeats(executorInfoList, whb));
- }
- return executorWhbs;
- }
-
- @Override
- public List<String> supervisors(IFn callback) {
- if (callback != null) {
- supervisorsCallback.set(callback);
- }
- return clusterState.get_children(Cluster.SUPERVISORS_SUBTREE, callback != null);
- }
-
- @Override
- public SupervisorInfo supervisorInfo(String supervisorId) {
- String path = Cluster.supervisorPath(supervisorId);
- return Cluster.maybeDeserialize(clusterState.get_data(path, false), SupervisorInfo.class);
- }
-
- @Override
- public void setupHeatbeats(String stormId) {
- clusterState.mkdirs(Cluster.workerbeatStormRoot(stormId), acls);
- }
-
- @Override
- public void teardownHeartbeats(String stormId) {
- try {
- clusterState.delete_worker_hb(Cluster.workerbeatStormRoot(stormId));
- } catch (Exception e) {
- if (Zookeeper.exceptionCause(KeeperException.class, e)) {
- // do nothing
- LOG.warn("Could not teardown heartbeats for {}.", stormId);
- } else {
- throw e;
- }
- }
- }
-
- @Override
- public void teardownTopologyErrors(String stormId) {
- try {
- clusterState.delete_node(Cluster.errorStormRoot(stormId));
- } catch (Exception e) {
- if (Zookeeper.exceptionCause(KeeperException.class, e)) {
- // do nothing
- LOG.warn("Could not teardown errors for {}.", stormId);
- } else {
- throw e;
- }
- }
- }
-
- @Override
- public List<String> heartbeatStorms() {
- return clusterState.get_worker_hb_children(Cluster.WORKERBEATS_SUBTREE, false);
- }
-
- @Override
- public List<String> errorTopologies() {
- return clusterState.get_children(Cluster.ERRORS_SUBTREE, false);
- }
-
- @Override
- public void setTopologyLogConfig(String stormId, LogConfig logConfig) {
- clusterState.set_data(Cluster.logConfigPath(stormId), Utils.serialize(logConfig), acls);
- }
-
- @Override
- public LogConfig topologyLogConfig(String stormId, IFn cb) {
- String path = Cluster.logConfigPath(stormId);
- return Cluster.maybeDeserialize(clusterState.get_data(path, cb != null), LogConfig.class);
- }
-
- @Override
- public void workerHeartbeat(String stormId, String node, Long port, ClusterWorkerHeartbeat info) {
- if (info != null) {
- String path = Cluster.workerbeatPath(stormId, node, port);
- clusterState.set_worker_hb(path, Utils.serialize(info), acls);
- }
- }
-
- @Override
- public void removeWorkerHeartbeat(String stormId, String node, Long port) {
- String path = Cluster.workerbeatPath(stormId, node, port);
- clusterState.delete_worker_hb(path);
- }
-
- @Override
- public void supervisorHeartbeat(String supervisorId, SupervisorInfo info) {
- String path = Cluster.supervisorPath(supervisorId);
- clusterState.set_ephemeral_node(path, Utils.serialize(info), acls);
- }
-
- // if znode exists and to be not on?, delete; if exists and on?, do nothing;
- // if not exists and to be on?, create; if not exists and not on?, do nothing;
- @Override
- public void workerBackpressure(String stormId, String node, Long port, boolean on) {
- String path = Cluster.backpressurePath(stormId, node, port);
- boolean existed = clusterState.node_exists(path, false);
- if (existed) {
- if (on == false)
- clusterState.delete_node(path);
-
- } else {
- if (on == true) {
- clusterState.set_ephemeral_node(path, null, acls);
- }
- }
- }
-
- // if the backpresure/storm-id dir is empty, this topology has throttle-on, otherwise not.
- @Override
- public boolean topologyBackpressure(String stormId, IFn callback) {
- if (callback != null) {
- backPressureCallback.put(stormId, callback);
- }
- String path = Cluster.backpressureStormRoot(stormId);
- List<String> childrens = clusterState.get_children(path, callback != null);
- return childrens.size() > 0;
-
- }
-
- @Override
- public void setupBackpressure(String stormId) {
- clusterState.mkdirs(Cluster.backpressureStormRoot(stormId), acls);
- }
-
- @Override
- public void removeWorkerBackpressure(String stormId, String node, Long port) {
- clusterState.delete_node(Cluster.backpressurePath(stormId, node, port));
- }
-
- @Override
- public void activateStorm(String stormId, StormBase stormBase) {
- String path = Cluster.stormPath(stormId);
- clusterState.set_data(path, Utils.serialize(stormBase), acls);
- }
-
- // maybe exit some questions for updateStorm
- @Override
- public void updateStorm(String stormId, StormBase newElems) {
-
- StormBase stormBase = stormBase(stormId, null);
- if (stormBase.get_component_executors() != null) {
-
- Map<String, Integer> newComponentExecutors = new HashMap<>();
- Map<String, Integer> componentExecutors = newElems.get_component_executors();
- //componentExecutors maybe be APersistentMap, which don't support put
- for (Map.Entry<String, Integer> entry : componentExecutors.entrySet()) {
- newComponentExecutors.put(entry.getKey(), entry.getValue());
- }
- for (Map.Entry<String, Integer> entry : stormBase.get_component_executors().entrySet()) {
- if (!componentExecutors.containsKey(entry.getKey())) {
- newComponentExecutors.put(entry.getKey(), entry.getValue());
- }
- }
- if (newComponentExecutors.size() > 0)
- newElems.set_component_executors(newComponentExecutors);
- }
-
- Map<String, DebugOptions> ComponentDebug = new HashMap<>();
- Map<String, DebugOptions> oldComponentDebug = stormBase.get_component_debug();
-
- Map<String, DebugOptions> newComponentDebug = newElems.get_component_debug();
-
- Set<String> debugOptionsKeys = oldComponentDebug.keySet();
- debugOptionsKeys.addAll(newComponentDebug.keySet());
- for (String key : debugOptionsKeys) {
- boolean enable = false;
- double samplingpct = 0;
- if (oldComponentDebug.containsKey(key)) {
- enable = oldComponentDebug.get(key).is_enable();
- samplingpct = oldComponentDebug.get(key).get_samplingpct();
- }
- if (newComponentDebug.containsKey(key)) {
- enable = newComponentDebug.get(key).is_enable();
- samplingpct += newComponentDebug.get(key).get_samplingpct();
- }
- DebugOptions debugOptions = new DebugOptions();
- debugOptions.set_enable(enable);
- debugOptions.set_samplingpct(samplingpct);
- ComponentDebug.put(key, debugOptions);
- }
- if (ComponentDebug.size() > 0) {
- newElems.set_component_debug(ComponentDebug);
- }
-
-
- if (StringUtils.isBlank(newElems.get_name())) {
- newElems.set_name(stormBase.get_name());
- }
- if (newElems.get_status() == null){
- newElems.set_status(stormBase.get_status());
- }
- if (newElems.get_num_workers() == 0){
- newElems.set_num_workers(stormBase.get_num_workers());
- }
- if (newElems.get_launch_time_secs() == 0) {
- newElems.set_launch_time_secs(stormBase.get_launch_time_secs());
- }
- if (StringUtils.isBlank(newElems.get_owner())) {
- newElems.set_owner(stormBase.get_owner());
- }
- if (newElems.get_topology_action_options() == null) {
- newElems.set_topology_action_options(stormBase.get_topology_action_options());
- }
- if (newElems.get_status() == null) {
- newElems.set_status(stormBase.get_status());
- }
- clusterState.set_data(Cluster.stormPath(stormId), Utils.serialize(newElems), acls);
- }
-
- @Override
- public void removeStormBase(String stormId) {
- clusterState.delete_node(Cluster.stormPath(stormId));
- }
-
- @Override
- public void setAssignment(String stormId, Assignment info) {
- clusterState.set_data(Cluster.assignmentPath(stormId), Utils.serialize(info), acls);
- }
-
- @Override
- public void setupBlobstore(String key, NimbusInfo nimbusInfo, Integer versionInfo) {
- String path = Cluster.blobstorePath(key) + Cluster.ZK_SEPERATOR + nimbusInfo.toHostPortString() + "-" + versionInfo;
- LOG.info("set-path: {}", path);
- clusterState.mkdirs(Cluster.blobstorePath(key), acls);
- clusterState.delete_node_blobstore(Cluster.blobstorePath(key), nimbusInfo.toHostPortString());
- clusterState.set_ephemeral_node(path, null, acls);
- }
-
- @Override
- public List<String> activeKeys() {
- return clusterState.get_children(Cluster.BLOBSTORE_SUBTREE, false);
- }
-
- // blobstore state
- @Override
- public List<String> blobstore(IFn callback) {
- if (callback != null) {
- blobstoreCallback.set(callback);
- }
- clusterState.sync_path(Cluster.BLOBSTORE_SUBTREE);
- return clusterState.get_children(Cluster.BLOBSTORE_SUBTREE, callback != null);
-
- }
-
- @Override
- public void removeStorm(String stormId) {
- clusterState.delete_node(Cluster.assignmentPath(stormId));
- clusterState.delete_node(Cluster.credentialsPath(stormId));
- clusterState.delete_node(Cluster.logConfigPath(stormId));
- clusterState.delete_node(Cluster.profilerConfigPath(stormId));
- removeStormBase(stormId);
- }
-
- @Override
- public void removeBlobstoreKey(String blobKey) {
- LOG.debug("remove key {}", blobKey);
- clusterState.delete_node(Cluster.blobstorePath(blobKey));
- }
-
- @Override
- public void removeKeyVersion(String blobKey) {
- clusterState.delete_node(Cluster.blobstoreMaxKeySequenceNumberPath(blobKey));
- }
-
- @Override
- public void reportError(String stormId, String componentId, String node, Integer port, String error) {
-
- try {
- String path = Cluster.errorPath(stormId, componentId);
- String lastErrorPath = Cluster.lastErrorPath(stormId, componentId);
- ErrorInfo errorInfo = new ErrorInfo(error, Time.currentTimeSecs());
- errorInfo.set_host(node);
- errorInfo.set_port(port.intValue());
- byte[] serData = Utils.serialize(errorInfo);
- clusterState.mkdirs(path, acls);
- clusterState.create_sequential(path + Cluster.ZK_SEPERATOR + "e", serData, acls);
- clusterState.set_data(lastErrorPath, serData, acls);
- List<String> childrens = clusterState.get_children(path, false);
-
- Collections.sort(childrens);
-
- while (childrens.size() >= 10) {
- clusterState.delete_node(path + Cluster.ZK_SEPERATOR + childrens.remove(0));
- }
- } catch (UnsupportedEncodingException e) {
- throw Utils.wrapInRuntime(e);
- }
- }
-
- @Override
- public List<ErrorInfo> errors(String stormId, String componentId) {
- List<ErrorInfo> errorInfos = new ArrayList<>();
- try {
- String path = Cluster.errorPath(stormId, componentId);
- if (clusterState.node_exists(path, false)) {
- List<String> childrens = clusterState.get_children(path, false);
- for (String child : childrens) {
- String childPath = path + Cluster.ZK_SEPERATOR + child;
- ErrorInfo errorInfo = Cluster.maybeDeserialize(clusterState.get_data(childPath, false), ErrorInfo.class);
- if (errorInfo != null)
- errorInfos.add(errorInfo);
- }
- }
- Collections.sort(errorInfos, new Comparator<ErrorInfo>() {
- public int compare(ErrorInfo arg0, ErrorInfo arg1) {
- return Integer.compare(arg0.get_error_time_secs(), arg1.get_error_time_secs());
- }
- });
- } catch (Exception e) {
- throw Utils.wrapInRuntime(e);
- }
-
- return errorInfos;
- }
-
- @Override
- public ErrorInfo lastError(String stormId, String componentId) {
- try {
- String path = Cluster.lastErrorPath(stormId, componentId);
- if (clusterState.node_exists(path, false)) {
- ErrorInfo errorInfo = Cluster.maybeDeserialize(clusterState.get_data(path, false), ErrorInfo.class);
- return errorInfo;
- }
- } catch (UnsupportedEncodingException e) {
- throw Utils.wrapInRuntime(e);
- }
- return null;
- }
-
- @Override
- public void setCredentials(String stormId, Credentials creds, Map topoConf) throws NoSuchAlgorithmException {
- List<ACL> aclList = Cluster.mkTopoOnlyAcls(topoConf);
- String path = Cluster.credentialsPath(stormId);
- clusterState.set_data(path, Utils.serialize(creds), aclList);
-
- }
-
- @Override
- public Credentials credentials(String stormId, IFn callback) {
- if (callback != null) {
- credentialsCallback.put(stormId, callback);
- }
- String path = Cluster.credentialsPath(stormId);
- return Cluster.maybeDeserialize(clusterState.get_data(path, callback != null), Credentials.class);
-
- }
-
- @Override
- public void disconnect() {
- clusterState.unregister(stateId);
- if (solo)
- clusterState.close();
- }
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
new file mode 100644
index 0000000..8ac0adc
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
@@ -0,0 +1,271 @@
+/**
+ * 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 org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.state.*;
+import org.apache.curator.framework.state.ConnectionState;
+import org.apache.storm.Config;
+import org.apache.storm.callback.DefaultWatcherCallBack;
+import org.apache.storm.callback.WatcherCallBack;
+import org.apache.storm.callback.ZKStateChangedCallback;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.zookeeper.Zookeeper;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class ZKStateStorage implements StateStorage {
+
+ private static Logger LOG = LoggerFactory.getLogger(ZKStateStorage.class);
+
+ private ConcurrentHashMap<String, ZKStateChangedCallback> callbacks = new ConcurrentHashMap<String, ZKStateChangedCallback>();
+ private CuratorFramework zkWriter;
+ private CuratorFramework zkReader;
+ private AtomicBoolean active;
+
+ private boolean isNimbus;
+ private Map authConf;
+ private Map<Object, Object> conf;
+
+ public ZKStateStorage(Map<Object, Object> conf, Map authConf, List<ACL> acls, ClusterStateContext context) throws Exception {
+ this.conf = conf;
+ this.authConf = authConf;
+ if (context.getDaemonType().equals(DaemonType.NIMBUS))
+ this.isNimbus = true;
+
+ // just mkdir STORM_ZOOKEEPER_ROOT dir
+ CuratorFramework zkTemp = mkZk();
+ String rootPath = String.valueOf(conf.get(Config.STORM_ZOOKEEPER_ROOT));
+ Zookeeper.mkdirs(zkTemp, rootPath, acls);
+ zkTemp.close();
+
+ active = new AtomicBoolean(true);
+ zkWriter = mkZk(new WatcherCallBack() {
+ @Override
+ public void execute(Watcher.Event.KeeperState state, Watcher.Event.EventType type, String path) {
+ if (active.get()) {
+ if (!(state.equals(Watcher.Event.KeeperState.SyncConnected))) {
+ LOG.warn("Received event {} : {}: {} with disconnected Zookeeper.", state, type, path);
+ } else {
+ LOG.info("Received event {} : {} : {}", state, type, path);
+ }
+
+ if (!type.equals(Watcher.Event.EventType.None)) {
+ for (Map.Entry<String, ZKStateChangedCallback> e : callbacks.entrySet()) {
+ ZKStateChangedCallback fn = e.getValue();
+ fn.changed(type, path);
+ }
+ }
+ }
+ }
+ });
+ if (isNimbus) {
+ zkReader = mkZk(new WatcherCallBack() {
+ @Override
+ public void execute(Watcher.Event.KeeperState state, Watcher.Event.EventType type, String path) {
+ if (active.get()) {
+ if (!(state.equals(Watcher.Event.KeeperState.SyncConnected))) {
+ LOG.warn("Received event {} : {}: {} with disconnected Zookeeper.", state, type, path);
+ } else {
+ LOG.debug("Received event {} : {} : {}", state, type, path);
+ }
+
+ if (!type.equals(Watcher.Event.EventType.None)) {
+ for (Map.Entry<String, ZKStateChangedCallback> e : callbacks.entrySet()) {
+ ZKStateChangedCallback fn = e.getValue();
+ fn.changed(type, path);
+ }
+ }
+ }
+ }
+ });
+ } else {
+ zkReader = zkWriter;
+ }
+
+ }
+
+ @SuppressWarnings("unchecked")
+ private CuratorFramework mkZk() throws IOException {
+ return Zookeeper.mkClient(conf, (List<String>) conf.get(Config.STORM_ZOOKEEPER_SERVERS), conf.get(Config.STORM_ZOOKEEPER_PORT), "",
+ new DefaultWatcherCallBack(), authConf);
+ }
+
+ @SuppressWarnings("unchecked")
+ private CuratorFramework mkZk(WatcherCallBack watcher) throws NumberFormatException, IOException {
+ return Zookeeper.mkClient(conf, (List<String>) conf.get(Config.STORM_ZOOKEEPER_SERVERS), conf.get(Config.STORM_ZOOKEEPER_PORT),
+ String.valueOf(conf.get(Config.STORM_ZOOKEEPER_ROOT)), watcher, authConf);
+ }
+
+ @Override
+ public void delete_node_blobstore(String path, String nimbusHostPortInfo) {
+
+ }
+
+ @Override
+ public String register(ZKStateChangedCallback callback) {
+ String id = UUID.randomUUID().toString();
+ this.callbacks.put(id, callback);
+ return id;
+ }
+
+ @Override
+ public void unregister(String id) {
+ this.callbacks.remove(id);
+ }
+
+ @Override
+ public String create_sequential(String path, byte[] data, List<ACL> acls) {
+ return Zookeeper.createNode(zkWriter, path, data, CreateMode.EPHEMERAL_SEQUENTIAL, acls);
+ }
+
+ @Override
+ public void mkdirs(String path, List<ACL> acls) {
+ Zookeeper.mkdirs(zkWriter, path, acls);
+ }
+
+ @Override
+ public void delete_node(String path) {
+ Zookeeper.deleteNode(zkWriter, path);
+ }
+
+ @Override
+ public void set_ephemeral_node(String path, byte[] data, List<ACL> acls) {
+ Zookeeper.mkdirs(zkWriter, parentPath(path), acls);
+ if (Zookeeper.exists(zkWriter, path, false)) {
+ try {
+ Zookeeper.setData(zkWriter, path, data);
+ } catch (Exception e) {
+ if (Utils.exceptionCauseIsInstanceOf(KeeperException.NoNodeException.class, e)) {
+ Zookeeper.createNode(zkWriter, path, data, CreateMode.EPHEMERAL, acls);
+ } else {
+ throw Utils.wrapInRuntime(e);
+ }
+ }
+
+ } else {
+ Zookeeper.createNode(zkWriter, path, data, CreateMode.EPHEMERAL, acls);
+ }
+ }
+
+ @Override
+ public Integer get_version(String path, boolean watch) throws Exception {
+ Integer ret = Zookeeper.getVersion(zkReader, path, watch);
+ return ret;
+ }
+
+ @Override
+ public boolean node_exists(String path, boolean watch) {
+ return Zookeeper.existsNode(zkWriter, path, watch);
+ }
+
+ @Override
+ public List<String> get_children(String path, boolean watch) {
+ return Zookeeper.getChildren(zkReader, path, watch);
+ }
+
+ @Override
+ public void close() {
+ this.active.set(false);
+ zkWriter.close();
+ if (isNimbus) {
+ zkReader.close();
+ }
+ }
+
+ @Override
+ public void set_data(String path, byte[] data, List<ACL> acls) {
+ if (Zookeeper.exists(zkWriter, path, false)) {
+ Zookeeper.setData(zkWriter, path, data);
+ } else {
+ Zookeeper.mkdirs(zkWriter, parentPath(path), acls);
+ Zookeeper.createNode(zkWriter, path, data, CreateMode.PERSISTENT, acls);
+ }
+ }
+
+ @Override
+ public byte[] get_data(String path, boolean watch) {
+ byte[] ret = null;
+
+ ret = Zookeeper.getData(zkReader, path, watch);
+
+ return ret;
+ }
+
+ @Override
+ public APersistentMap get_data_with_version(String path, boolean watch) {
+ return Zookeeper.getDataWithVersion(zkReader, path, watch);
+ }
+
+ @Override
+ public void set_worker_hb(String path, byte[] data, List<ACL> acls) {
+ set_data(path, data, acls);
+ }
+
+ @Override
+ public byte[] get_worker_hb(String path, boolean watch) {
+ return Zookeeper.getData(zkReader, path, watch);
+ }
+
+ @Override
+ public List<String> get_worker_hb_children(String path, boolean watch) {
+ return get_children(path, watch);
+ }
+
+ @Override
+ public void delete_worker_hb(String path) {
+ delete_node(path);
+ }
+
+ @Override
+ public void add_listener(final ConnectionStateListener listener) {
+ Zookeeper.addListener(zkReader, new ConnectionStateListener() {
+ @Override
+ public void stateChanged(CuratorFramework curatorFramework, ConnectionState connectionState) {
+ listener.stateChanged(curatorFramework, connectionState);
+ }
+ });
+ }
+
+ @Override
+ public void sync_path(String path) {
+ Zookeeper.syncPath(zkWriter, path);
+ }
+
+ // To be remove when finished port Util.clj
+ public static String parentPath(String path) {
+ List<String> toks = Zookeeper.tokenizePath(path);
+ int size = toks.size();
+ if (size > 0) {
+ toks.remove(size - 1);
+ }
+ return Zookeeper.toksToPath(toks);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
new file mode 100644
index 0000000..19b04f2
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
@@ -0,0 +1,36 @@
+/**
+ * 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 org.apache.storm.utils.Utils;
+import org.apache.zookeeper.data.ACL;
+
+import java.util.List;
+
+public class ZKStateStorageFactory implements StateStorageFactory{
+
+ @Override
+ public StateStorage mkState(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) {
+ try {
+ return new ZKStateStorage(config, auth_conf, acls, context);
+ }catch (Exception e){
+ throw Utils.wrapInRuntime(e);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedCluster.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedCluster.java b/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedCluster.java
index 5d67a54..2f1440c 100644
--- a/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedCluster.java
+++ b/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedCluster.java
@@ -16,16 +16,16 @@
*/
package org.apache.storm.testing.staticmocking;
-import org.apache.storm.cluster.Cluster;
+import org.apache.storm.cluster.ClusterUtils;
public class MockedCluster implements AutoCloseable {
- public MockedCluster(Cluster inst) {
- Cluster.setInstance(inst);
+ public MockedCluster(ClusterUtils inst) {
+ ClusterUtils.setInstance(inst);
}
@Override
public void close() throws Exception {
- Cluster.resetInstance();
+ ClusterUtils.resetInstance();
}
}
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java b/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
index f1c7f32..c280515 100644
--- a/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
+++ b/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
@@ -86,19 +86,23 @@ public class Zookeeper {
_instance = INSTANCE;
}
- public static CuratorFramework mkClient(Map conf, List<String> servers, Object port, String root) {
- return mkClient(conf, servers, port, root, new DefaultWatcherCallBack());
+ public CuratorFramework mkClientImpl(Map conf, List<String> servers, Object port, String root) {
+ return mkClientImpl(conf, servers, port, root, new DefaultWatcherCallBack());
}
- public static CuratorFramework mkClient(Map conf, List<String> servers, Object port, Map authConf) {
- return mkClient(conf, servers, port, "", new DefaultWatcherCallBack(), authConf);
+ public CuratorFramework mkClientImpl(Map conf, List<String> servers, Object port, Map authConf) {
+ return mkClientImpl(conf, servers, port, "", new DefaultWatcherCallBack(), authConf);
}
- public static CuratorFramework mkClient(Map conf, List<String> servers, Object port, String root, Map authConf) {
- return mkClient(conf, servers, port, root, new DefaultWatcherCallBack(), authConf);
+ public CuratorFramework mkClientImpl(Map conf, List<String> servers, Object port, String root, Map authConf) {
+ return mkClientImpl(conf, servers, port, root, new DefaultWatcherCallBack(), authConf);
}
public static CuratorFramework mkClient(Map conf, List<String> servers, Object port, String root, final WatcherCallBack watcher, Map authConf) {
+ return _instance.mkClientImpl(conf, servers, port, root, watcher, authConf);
+ }
+
+ public CuratorFramework mkClientImpl(Map conf, List<String> servers, Object port, String root, final WatcherCallBack watcher, Map authConf) {
CuratorFramework fk;
if (authConf != null) {
fk = Utils.newCurator(conf, servers, port, root, new ZookeeperAuthInfo(authConf));
@@ -124,8 +128,8 @@ public class Zookeeper {
*
* @return
*/
- public static CuratorFramework mkClient(Map conf, List<String> servers, Object port, String root, final WatcherCallBack watcher) {
- return mkClient(conf, servers, port, root, watcher, null);
+ public CuratorFramework mkClientImpl(Map conf, List<String> servers, Object port, String root, final WatcherCallBack watcher) {
+ return mkClientImpl(conf, servers, port, root, watcher, null);
}
public static String createNode(CuratorFramework zk, String path, byte[] data, org.apache.zookeeper.CreateMode mode, List<ACL> acls) {
@@ -347,7 +351,7 @@ public class Zookeeper {
protected ILeaderElector zkLeaderElectorImpl(Map conf) throws UnknownHostException {
List<String> servers = (List<String>) conf.get(Config.STORM_ZOOKEEPER_SERVERS);
Object port = conf.get(Config.STORM_ZOOKEEPER_PORT);
- CuratorFramework zk = mkClient(conf, servers, port, "", conf);
+ CuratorFramework zk = mkClientImpl(conf, servers, port, "", conf);
String leaderLockPath = conf.get(Config.STORM_ZOOKEEPER_ROOT) + "/leader-lock";
String id = NimbusInfo.fromConf(conf).toHostPortString();
AtomicReference<LeaderLatch> leaderLatchAtomicReference = new AtomicReference<>(new LeaderLatch(zk, leaderLockPath, id));
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/integration/org/apache/storm/integration_test.clj b/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
index d374511..d4fab3f 100644
--- a/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
+++ b/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
@@ -21,7 +21,7 @@
(:import [org.apache.storm.testing TestWordCounter TestWordSpout TestGlobalCount
TestAggregatesCounter TestConfBolt AckFailMapTracker AckTracker TestPlannerSpout])
(:import [org.apache.storm.tuple Fields])
- (:import [org.apache.storm.cluster StormZkClusterState])
+ (:import [org.apache.storm.cluster StormClusterStateImpl])
(:use [org.apache.storm testing config clojure util converter])
(:use [org.apache.storm.daemon common])
(:require [org.apache.storm [thrift :as thrift]]))
@@ -576,7 +576,7 @@
(:topology tracked))
_ (advance-cluster-time cluster 11)
storm-id (get-storm-id state "test-errors")
- errors-count (fn [] (count (clojurify-error (.errors state storm-id "2"))))]
+ errors-count (fn [] (count (.errors state storm-id "2")))]
(is (nil? (clojurify-error (.lastError state storm-id "2"))))
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/test/clj/org/apache/storm/cluster_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/cluster_test.clj b/storm-core/test/clj/org/apache/storm/cluster_test.clj
index d0b9882..fa34355 100644
--- a/storm-core/test/clj/org/apache/storm/cluster_test.clj
+++ b/storm-core/test/clj/org/apache/storm/cluster_test.clj
@@ -23,9 +23,10 @@
(:import [org.mockito.exceptions.base MockitoAssertionError])
(:import [org.apache.curator.framework CuratorFramework CuratorFrameworkFactory CuratorFrameworkFactory$Builder])
(:import [org.apache.storm.utils Utils TestUtils ZookeeperAuthInfo ConfigUtils])
- (:import [org.apache.storm.cluster ClusterState DistributedClusterState ClusterStateContext StormZkClusterState])
+ (:import [org.apache.storm.cluster StateStorage ZKStateStorage ClusterStateContext StormClusterStateImpl ClusterUtils])
(:import [org.apache.storm.zookeeper Zookeeper])
- (:import [org.apache.storm.testing.staticmocking MockedZookeeper])
+ (:import [org.apache.storm.callback ZKStateChangedCallback])
+ (:import [org.apache.storm.testing.staticmocking MockedZookeeper MockedCluster])
(:require [conjure.core])
(:use [conjure core])
(:use [clojure test])
@@ -33,18 +34,18 @@
(defn mk-config [zk-port]
(merge (clojurify-structure (ConfigUtils/readStormConfig))
- {STORM-ZOOKEEPER-PORT zk-port
- STORM-ZOOKEEPER-SERVERS ["localhost"]}))
+ {STORM-ZOOKEEPER-PORT zk-port
+ STORM-ZOOKEEPER-SERVERS ["localhost"]}))
(defn mk-state
([zk-port] (let [conf (mk-config zk-port)]
- (DistributedClusterState. conf conf nil (ClusterStateContext.))))
+ (ClusterUtils/mkDistributedClusterState conf conf nil (ClusterStateContext.))))
([zk-port cb]
- (let [ret (mk-state zk-port)]
- (.register ret cb)
- ret )))
+ (let [ret (mk-state zk-port)]
+ (.register ret cb)
+ ret)))
-(defn mk-storm-state [zk-port] (StormZkClusterState. (mk-config zk-port) nil (ClusterStateContext.)))
+(defn mk-storm-state [zk-port] (ClusterUtils/mkStormClusterState (mk-config zk-port) nil (ClusterStateContext.)))
(deftest test-basics
(with-inprocess-zookeeper zk-port
@@ -99,24 +100,27 @@
(defn mk-callback-tester []
(let [last (atom nil)
- cb (fn [type path]
- (reset! last {:type type :path path}))]
+ cb (reify
+ ZKStateChangedCallback
+ (changed
+ [this type path]
+ (reset! last {:type type :path path})))]
[last cb]
))
(defn read-and-reset! [aatom]
(let [time (System/currentTimeMillis)]
- (loop []
- (if-let [val @aatom]
- (do
- (reset! aatom nil)
- val)
- (do
- (when (> (- (System/currentTimeMillis) time) 30000)
- (throw (RuntimeException. "Waited too long for atom to change state")))
- (Thread/sleep 10)
- (recur))
- ))))
+ (loop []
+ (if-let [val @aatom]
+ (do
+ (reset! aatom nil)
+ val)
+ (do
+ (when (> (- (System/currentTimeMillis) time) 30000)
+ (throw (RuntimeException. "Waited too long for atom to change state")))
+ (Thread/sleep 10)
+ (recur))
+ ))))
(deftest test-callbacks
(with-inprocess-zookeeper zk-port
@@ -189,35 +193,35 @@
(is (= #{"storm1" "storm3"} (set (.assignments state nil))))
(is (= assignment2 (clojurify-assignment (.assignmentInfo state "storm1" nil))))
(is (= assignment1 (clojurify-assignment (.assignmentInfo state "storm3" nil))))
-
- (is (= [] (.active-storms state)))
+
+ (is (= [] (.activeStorms state)))
(.activateStorm state "storm1" (thriftify-storm-base base1))
- (is (= ["storm1"] (.active-storms state)))
+ (is (= ["storm1"] (.activeStorms state)))
(is (= base1 (clojurify-storm-base (.stormBase state "storm1" nil))))
(is (= nil (clojurify-storm-base (.stormBase state "storm2" nil))))
(.activateStorm state "storm2" (thriftify-storm-base base2))
(is (= base1 (clojurify-storm-base (.stormBase state "storm1" nil))))
(is (= base2 (clojurify-storm-base (.stormBase state "storm2" nil))))
- (is (= #{"storm1" "storm2"} (set (.active-storms state))))
+ (is (= #{"storm1" "storm2"} (set (.activeStorms state))))
(.removeStormBase state "storm1")
(is (= base2 (clojurify-storm-base (.stormBase state "storm2" nil))))
- (is (= #{"storm2"} (set (.active-storms state))))
+ (is (= #{"storm2"} (set (.activeStorms state))))
(is (nil? (clojurify-crdentials (.credentials state "storm1" nil))))
- (.setCredentials! state "storm1" (thriftify-credentials {"a" "a"}) {})
+ (.setCredentials state "storm1" (thriftify-credentials {"a" "a"}) {})
(is (= {"a" "a"} (clojurify-crdentials (.credentials state "storm1" nil))))
(.setCredentials state "storm1" (thriftify-credentials {"b" "b"}) {})
(is (= {"b" "b"} (clojurify-crdentials (.credentials state "storm1" nil))))
- (is (= [] (.blobstoreInfo state nil)))
- (.setupBlobstore state "key1" nimbusInfo1 "1")
- (is (= ["key1"] (.blobstoreInfo state nil)))
+ (is (= [] (.blobstoreInfo state "")))
+ (.setupBlobstore state "key1" nimbusInfo1 (Integer/parseInt "1"))
+ (is (= ["key1"] (.blobstoreInfo state "")))
(is (= [(str (.toHostPortString nimbusInfo1) "-1")] (.blobstoreInfo state "key1")))
- (.setupBlobstore state "key1" nimbusInfo2 "1")
+ (.setupBlobstore state "key1" nimbusInfo2 (Integer/parseInt "1"))
(is (= #{(str (.toHostPortString nimbusInfo1) "-1")
(str (.toHostPortString nimbusInfo2) "-1")} (set (.blobstoreInfo state "key1"))))
(.removeBlobstoreKey state "key1")
- (is (= [] (.blobstoreInfo state nil)))
+ (is (= [] (.blobstoreInfo state "")))
(is (= [] (.nimbuses state)))
(.addNimbusHost state "nimbus1:port" nimbusSummary1)
@@ -230,11 +234,10 @@
)))
(defn- validate-errors! [state storm-id component errors-list]
- (let [errors (clojurify-error (.errors state storm-id component))]
- ;;(println errors)
+ (let [errors (map clojurify-error (.errors state storm-id component))]
(is (= (count errors) (count errors-list)))
(doseq [[error target] (map vector errors errors-list)]
- (when-not (.contains (:error error) target)
+ (when-not (.contains (:error error) target)
(println target " => " (:error error)))
(is (.contains (:error error) target))
)))
@@ -257,8 +260,9 @@
(.reportError state "a" "2" (local-hostname) 6700 (stringify-error (IllegalArgumentException.)))
(advance-time-secs! 2))
(validate-errors! state "a" "2" (concat (repeat 5 "IllegalArgumentException")
- (repeat 5 "RuntimeException")
- ))
+ (repeat 5 "RuntimeException")
+ ))
+
(.disconnect state)
))))
@@ -285,23 +289,23 @@
(with-inprocess-zookeeper zk-port
(let [builder (Mockito/mock CuratorFrameworkFactory$Builder)
conf (merge
- (mk-config zk-port)
- {STORM-ZOOKEEPER-CONNECTION-TIMEOUT 10
- STORM-ZOOKEEPER-SESSION-TIMEOUT 10
- STORM-ZOOKEEPER-RETRY-INTERVAL 5
- STORM-ZOOKEEPER-RETRY-TIMES 2
- STORM-ZOOKEEPER-RETRY-INTERVAL-CEILING 15
- STORM-ZOOKEEPER-AUTH-SCHEME "digest"
- STORM-ZOOKEEPER-AUTH-PAYLOAD "storm:thisisapoorpassword"})]
+ (mk-config zk-port)
+ {STORM-ZOOKEEPER-CONNECTION-TIMEOUT 10
+ STORM-ZOOKEEPER-SESSION-TIMEOUT 10
+ STORM-ZOOKEEPER-RETRY-INTERVAL 5
+ STORM-ZOOKEEPER-RETRY-TIMES 2
+ STORM-ZOOKEEPER-RETRY-INTERVAL-CEILING 15
+ STORM-ZOOKEEPER-AUTH-SCHEME "digest"
+ STORM-ZOOKEEPER-AUTH-PAYLOAD "storm:thisisapoorpassword"})]
(. (Mockito/when (.connectString builder (Mockito/anyString))) (thenReturn builder))
(. (Mockito/when (.connectionTimeoutMs builder (Mockito/anyInt))) (thenReturn builder))
(. (Mockito/when (.sessionTimeoutMs builder (Mockito/anyInt))) (thenReturn builder))
(TestUtils/testSetupBuilder builder (str zk-port "/") conf (ZookeeperAuthInfo. conf))
(is (nil?
- (try
- (. (Mockito/verify builder) (authorization "digest" (.getBytes (conf STORM-ZOOKEEPER-AUTH-PAYLOAD))))
- (catch MockitoAssertionError e
- e)))))))
+ (try
+ (. (Mockito/verify builder) (authorization "digest" (.getBytes (conf STORM-ZOOKEEPER-AUTH-PAYLOAD))))
+ (catch MockitoAssertionError e
+ e)))))))
(deftest test-storm-state-callbacks
;; TODO finish
@@ -309,13 +313,17 @@
(deftest test-cluster-state-default-acls
(testing "The default ACLs are empty."
- (let [zk-mock (Mockito/mock Zookeeper)]
+ (let [zk-mock (Mockito/mock Zookeeper)
+ curator-frameworke (reify CuratorFramework (^void close [this] nil))]
;; No need for when clauses because we just want to return nil
(with-open [_ (MockedZookeeper. zk-mock)]
- (. (Mockito/when (Mockito/mock Zookeeper)) (thenReturn (reify CuratorFramework (^void close [this] nil))))
- (. (Mockito/when (Mockito/mock DistributedClusterState)) (thenReturn {}))
- (. (Mockito/when (Mockito/mock StormZkClusterState)) (thenReturn (reify ClusterState
- (register [this callback] nil)
- (mkdirs [this path acls] nil))))
- (.mkdirsImpl (Mockito/verify zk-mock (Mockito/times 1)) (Mockito/any) (Mockito/anyString) (Mockito/eq nil))))))
-
+ (. (Mockito/when (.mkClientImpl zk-mock (Mockito/anyMap) (Mockito/anyList) (Mockito/any) (Mockito/anyString) (Mockito/any) (Mockito/anyMap))) (thenReturn curator-frameworke))
+ (ClusterUtils/mkDistributedClusterState {} nil nil (ClusterStateContext.))
+ (.mkdirsImpl (Mockito/verify zk-mock (Mockito/times 1)) (Mockito/any) (Mockito/anyString) (Mockito/eq nil))))
+ (let [distributed-state-storage (reify StateStorage
+ (register [this callback] nil)
+ (mkdirs [this path acls] nil))
+ cluster-utils (Mockito/mock ClusterUtils)]
+ (with-open [mocked-cluster (MockedCluster. cluster-utils)]
+ (. (Mockito/when (.mkDistributedClusterStateImpl cluster-utils (Mockito/any) (Mockito/any) (Mockito/eq nil) (Mockito/any))) (thenReturn distributed-state-storage))
+ (ClusterUtils/mkStormClusterState {} nil (ClusterStateContext.))))))
\ No newline at end of file
[26/27] storm git commit: Merge branch 'ClusterUtils' of
https://github.com/hustfxj/storm into STORM-1273
Posted by bo...@apache.org.
Merge branch 'ClusterUtils' of https://github.com/hustfxj/storm into STORM-1273
STORM-1273: port backtype.storm.cluster to java
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/1b5d9a85
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/1b5d9a85
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/1b5d9a85
Branch: refs/heads/master
Commit: 1b5d9a85021ff4c7bb8e95a55a225277babb2378
Parents: 11232b5 0af6799
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Wed Feb 24 09:45:14 2016 -0600
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Wed Feb 24 09:45:14 2016 -0600
----------------------------------------------------------------------
conf/defaults.yaml | 2 +-
.../src/clj/org/apache/storm/thrift.clj | 2 +-
storm-core/src/clj/org/apache/storm/cluster.clj | 700 -------------------
.../cluster_state/zookeeper_state_factory.clj | 165 -----
.../clj/org/apache/storm/command/heartbeats.clj | 6 +-
.../apache/storm/command/shell_submission.clj | 2 +-
.../src/clj/org/apache/storm/converter.clj | 23 +-
.../src/clj/org/apache/storm/daemon/common.clj | 12 +-
.../clj/org/apache/storm/daemon/executor.clj | 13 +-
.../src/clj/org/apache/storm/daemon/nimbus.clj | 161 +++--
.../clj/org/apache/storm/daemon/supervisor.clj | 38 +-
.../src/clj/org/apache/storm/daemon/worker.clj | 59 +-
.../clj/org/apache/storm/internal/thrift.clj | 2 +-
.../storm/pacemaker/pacemaker_state_factory.clj | 141 ----
storm-core/src/clj/org/apache/storm/stats.clj | 3 +-
storm-core/src/clj/org/apache/storm/testing.clj | 12 +-
storm-core/src/clj/org/apache/storm/ui/core.clj | 2 +-
storm-core/src/clj/org/apache/storm/util.clj | 11 +
.../src/clj/org/apache/storm/zookeeper.clj | 74 --
.../jvm/org/apache/storm/callback/Callback.java | 23 -
.../storm/callback/ZKStateChangedCallback.java | 25 +
.../org/apache/storm/cluster/ClusterState.java | 217 ------
.../storm/cluster/ClusterStateContext.java | 2 +-
.../storm/cluster/ClusterStateFactory.java | 28 -
.../org/apache/storm/cluster/ClusterUtils.java | 244 +++++++
.../org/apache/storm/cluster/ExecutorBeat.java | 44 ++
.../org/apache/storm/cluster/IStateStorage.java | 222 ++++++
.../storm/cluster/IStormClusterState.java | 122 ++++
.../storm/cluster/PaceMakerStateStorage.java | 216 ++++++
.../cluster/PaceMakerStateStorageFactory.java | 64 ++
.../storm/cluster/StateStorageFactory.java | 28 +
.../storm/cluster/StormClusterStateImpl.java | 686 ++++++++++++++++++
.../apache/storm/cluster/ZKStateStorage.java | 244 +++++++
.../storm/cluster/ZKStateStorageFactory.java | 36 +
.../apache/storm/pacemaker/PacemakerClient.java | 6 +-
.../testing/staticmocking/MockedCluster.java | 31 +
.../MockedPaceMakerStateStorageFactory.java | 32 +
.../org/apache/storm/zookeeper/Zookeeper.java | 77 +-
.../org/apache/storm/integration_test.clj | 13 +-
.../test/clj/org/apache/storm/cluster_test.clj | 202 +++---
.../storm/messaging/netty_integration_test.clj | 1 +
.../test/clj/org/apache/storm/nimbus_test.clj | 158 +++--
.../storm/pacemaker_state_factory_test.clj | 121 ++--
.../storm/security/auth/nimbus_auth_test.clj | 3 +-
.../clj/org/apache/storm/supervisor_test.clj | 163 +++--
45 files changed, 2554 insertions(+), 1882 deletions(-)
----------------------------------------------------------------------
[18/27] storm git commit: removed any clojure internals
Posted by bo...@apache.org.
removed any clojure internals
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/afcd0c6c
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/afcd0c6c
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/afcd0c6c
Branch: refs/heads/master
Commit: afcd0c6c53aca7f99d39f10b91a7b45fda424fe5
Parents: d546387
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Fri Feb 19 15:21:48 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Fri Feb 19 15:22:22 2016 +0800
----------------------------------------------------------------------
.../src/clj/org/apache/storm/converter.clj | 9 +++
.../src/clj/org/apache/storm/daemon/nimbus.clj | 4 +-
.../clj/org/apache/storm/daemon/supervisor.clj | 8 +-
.../src/clj/org/apache/storm/daemon/worker.clj | 6 +-
storm-core/src/clj/org/apache/storm/testing.clj | 2 +-
.../org/apache/storm/cluster/ClusterUtils.java | 41 +++-------
.../org/apache/storm/cluster/ExecutorBeat.java | 44 +++++++++++
.../org/apache/storm/cluster/IStateStorage.java | 11 ++-
.../storm/cluster/IStormClusterState.java | 24 +++---
.../storm/cluster/PaceMakerStateStorage.java | 4 +-
.../cluster/PaceMakerStateStorageFactory.java | 12 +--
.../storm/cluster/StateStorageFactory.java | 6 +-
.../storm/cluster/StormClusterStateImpl.java | 78 ++++++++++----------
.../apache/storm/cluster/ZKStateStorage.java | 3 +-
.../storm/cluster/ZKStateStorageFactory.java | 4 +-
.../org/apache/storm/zookeeper/Zookeeper.java | 46 +++---------
16 files changed, 156 insertions(+), 146 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/clj/org/apache/storm/converter.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/converter.clj b/storm-core/src/clj/org/apache/storm/converter.clj
index 18647b1..c845cd4 100644
--- a/storm-core/src/clj/org/apache/storm/converter.clj
+++ b/storm-core/src/clj/org/apache/storm/converter.clj
@@ -18,6 +18,7 @@
StormBase TopologyStatus ClusterWorkerHeartbeat ExecutorInfo ErrorInfo Credentials RebalanceOptions KillOptions
TopologyActionOptions DebugOptions ProfileRequest]
[org.apache.storm.utils Utils])
+ (:import [org.apache.storm.cluster ExecutorBeat])
(:use [org.apache.storm util stats log])
(:require [org.apache.storm.daemon [common :as common]]))
@@ -238,6 +239,14 @@
}
{}))
+(defn clojurify-zk-executor-hb [^ExecutorBeat executor-hb]
+ (if executor-hb
+ {:stats (.getStats executor-hb)
+ :uptime (.getUptime executor-hb)
+ :time-secs (.getTimeSecs executor-hb)
+ }
+ {}))
+
(defn thriftify-zk-worker-hb [worker-hb]
(if (not-empty (filter second (:executor-stats worker-hb)))
(doto (ClusterWorkerHeartbeat.)
http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index 6bdbdc0..beb6639 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@ -591,8 +591,8 @@
(let [storm-cluster-state (:storm-cluster-state nimbus)
executor-beats (let [executor-stats-java-map (.executorBeats storm-cluster-state storm-id (.get_executor_node_port (thriftify-assignment existing-assignment)))
executor-stats-clojurify (clojurify-structure executor-stats-java-map)]
- (->> (dofor [[^ExecutorInfo executor-info executor-heartbeat] executor-stats-clojurify]
- {[(.get_task_start executor-info) (.get_task_end executor-info)] executor-heartbeat})
+ (->> (dofor [[^ExecutorInfo executor-info ^ExecutorBeat executor-heartbeat] executor-stats-clojurify]
+ {[(.get_task_start executor-info) (.get_task_end executor-info)] (clojurify-zk-executor-hb executor-heartbeat)})
(apply merge)))
cache (update-heartbeat-cache (@(:heartbeats-cache nimbus) storm-id)
http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
index c1f058f..58f6291 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
@@ -20,7 +20,7 @@
ConfigUtils]
[org.apache.storm.daemon Shutdownable]
[org.apache.storm Constants]
- [org.apache.storm.cluster ClusterStateContext DaemonType StormClusterStateImpl ClusterUtils]
+ [org.apache.storm.cluster ClusterStateContext DaemonType StormClusterStateImpl ClusterUtils IStateStorage]
[java.net JarURLConnection]
[java.net URI URLDecoder]
[org.apache.commons.io FileUtils])
@@ -69,8 +69,8 @@
(if (= assignment-version recorded-version)
{sid (get assignment-versions sid)}
(let [thriftify-assignment-version (.assignmentInfoWithVersion storm-cluster-state sid callback)
- assignment (clojurify-assignment (:data thriftify-assignment-version))]
- {sid {:data assignment :version (:version thriftify-assignment-version)}}))
+ assignment (clojurify-assignment (.get thriftify-assignment-version (IStateStorage/DATA)))]
+ {sid {:data assignment :version (.get thriftify-assignment-version (IStateStorage/VERSION))}}))
{sid nil})))
(apply merge)
(filter-val not-nil?))
@@ -1184,7 +1184,7 @@
(.readBlobTo blob-store (ConfigUtils/masterStormConfKey storm-id) (FileOutputStream. (ConfigUtils/supervisorStormConfPath tmproot)) nil)
(finally
(.shutdown blob-store)))
- (try (FileUtils/moveDirectory (File. tmproot) (File. stormroot)) (catch Exception e))
+ (FileUtils/moveDirectory (File. tmproot) (File. stormroot))
(setup-storm-code-dir conf (clojurify-structure (ConfigUtils/readSupervisorStormConf conf storm-id)) stormroot)
(let [classloader (.getContextClassLoader (Thread/currentThread))
http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/clj/org/apache/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/worker.clj b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
index b80cd9e..395be23 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
@@ -38,7 +38,7 @@
(:import [org.apache.storm.task WorkerTopologyContext])
(:import [org.apache.storm Constants])
(:import [org.apache.storm.security.auth AuthUtils])
- (:import [org.apache.storm.cluster ClusterStateContext DaemonType ZKStateStorage StormClusterStateImpl ClusterUtils])
+ (:import [org.apache.storm.cluster ClusterStateContext DaemonType ZKStateStorage StormClusterStateImpl ClusterUtils IStateStorage])
(:import [javax.security.auth Subject])
(:import [java.security PrivilegedExceptionAction])
(:import [org.apache.logging.log4j LogManager])
@@ -381,8 +381,8 @@
(let [version (.assignmentVersion storm-cluster-state storm-id callback)
assignment (if (= version (:version (get @(:assignment-versions worker) storm-id)))
(:data (get @(:assignment-versions worker) storm-id))
- (let [java-assignment (.assignmentInfoWithVersion storm-cluster-state storm-id callback)
- new-assignment {:data (clojurify-assignment (:data java-assignment)) :version version}]
+ (let [thriftify-assignment-version (.assignmentInfoWithVersion storm-cluster-state storm-id callback)
+ new-assignment {:data (clojurify-assignment (.get thriftify-assignment-version (IStateStorage/DATA))) :version version}]
(swap! (:assignment-versions worker) assoc storm-id new-assignment)
(:data new-assignment)))
my-assignment (-> assignment
http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/clj/org/apache/storm/testing.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/testing.clj b/storm-core/src/clj/org/apache/storm/testing.clj
index eef7754..3dee54b 100644
--- a/storm-core/src/clj/org/apache/storm/testing.clj
+++ b/storm-core/src/clj/org/apache/storm/testing.clj
@@ -448,7 +448,7 @@
component->tasks)
task-ids (apply concat (vals component->tasks))
assignment (clojurify-assignment (.assignmentInfo state storm-id nil))
- taskbeats (.taskbeats state storm-id (:task->node+port assignment)) ;hava question?
+ taskbeats (.taskbeats state storm-id (:task->node+port assignment))
heartbeats (dofor [id task-ids] (get taskbeats id))
stats (dofor [hb heartbeats] (if hb (stat-key (:stats hb)) 0))]
(reduce + stats)))
http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java b/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
index 0c663f0..aae4231 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
@@ -17,9 +17,6 @@
*/
package org.apache.storm.cluster;
-import clojure.lang.APersistentMap;
-import clojure.lang.PersistentArrayMap;
-import clojure.lang.RT;
import org.apache.storm.Config;
import org.apache.storm.generated.ClusterWorkerHeartbeat;
import org.apache.storm.generated.ExecutorInfo;
@@ -192,14 +189,15 @@ public class ClusterUtils {
* @param workerHeartbeat
* @return
*/
- public static Map<ExecutorInfo, APersistentMap> convertExecutorBeats(List<ExecutorInfo> executors, ClusterWorkerHeartbeat workerHeartbeat) {
- Map<ExecutorInfo, APersistentMap> executorWhb = new HashMap<>();
+ public static Map<ExecutorInfo, ExecutorBeat> convertExecutorBeats(List<ExecutorInfo> executors, ClusterWorkerHeartbeat workerHeartbeat) {
+ Map<ExecutorInfo, ExecutorBeat> executorWhb = new HashMap<>();
Map<ExecutorInfo, ExecutorStats> executorStatsMap = workerHeartbeat.get_executor_stats();
for (ExecutorInfo executor : executors) {
if (executorStatsMap.containsKey(executor)) {
- APersistentMap executorBeat =
- new PersistentArrayMap(new Object[] { RT.keyword(null, "time-secs"), workerHeartbeat.get_time_secs(), RT.keyword(null, "uptime"),
- workerHeartbeat.get_uptime_secs(), RT.keyword(null, "stats"), workerHeartbeat.get_executor_stats().get(executor) });
+ int time = workerHeartbeat.get_time_secs();
+ int uptime = workerHeartbeat.get_uptime_secs();
+ ExecutorStats executorStats = workerHeartbeat.get_executor_stats().get(executor);
+ ExecutorBeat executorBeat = new ExecutorBeat(time, uptime, executorStats);
executorWhb.put(executor, executorBeat);
}
}
@@ -210,13 +208,13 @@ public class ClusterUtils {
if (stateStorage instanceof IStateStorage) {
return new StormClusterStateImpl((IStateStorage) stateStorage, acls, context, false);
} else {
- IStateStorage Storage = _instance.mkStateStorageImpl((APersistentMap) stateStorage, (APersistentMap) stateStorage, acls, context);
+ IStateStorage Storage = _instance.mkStateStorageImpl((Map) stateStorage, (Map) stateStorage, acls, context);
return new StormClusterStateImpl(Storage, acls, context, true);
}
}
- public IStateStorage mkStateStorageImpl(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
+ public IStateStorage mkStateStorageImpl(Map config, Map auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
String className = null;
IStateStorage stateStorage = null;
if (config.get(Config.STORM_CLUSTER_STATE_STORE) != null) {
@@ -230,7 +228,7 @@ public class ClusterUtils {
return stateStorage;
}
- public static IStateStorage mkStateStorage(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
+ public static IStateStorage mkStateStorage(Map config, Map auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
return _instance.mkStateStorageImpl(config, auth_conf, acls, context);
}
@@ -238,26 +236,7 @@ public class ClusterUtils {
return _instance.mkStormClusterStateImpl(StateStorage, acls, context);
}
- // TO be remove
- public static <K, V> HashMap<V, List<K>> reverseMap(Map<K, V> map) {
- HashMap<V, List<K>> rtn = new HashMap<V, List<K>>();
- if (map == null) {
- return rtn;
- }
- for (Map.Entry<K, V> entry : map.entrySet()) {
- K key = entry.getKey();
- V val = entry.getValue();
- List<K> list = rtn.get(val);
- if (list == null) {
- list = new ArrayList<K>();
- rtn.put(entry.getValue(), list);
- }
- list.add(key);
- }
- return rtn;
- }
-
- public static String StringifyError(Throwable error) {
+ public static String stringifyError(Throwable error) {
String errorString = null;
StringWriter result = null;
PrintWriter printWriter = null;
http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/jvm/org/apache/storm/cluster/ExecutorBeat.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ExecutorBeat.java b/storm-core/src/jvm/org/apache/storm/cluster/ExecutorBeat.java
new file mode 100644
index 0000000..b32615e
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ExecutorBeat.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 org.apache.storm.cluster;
+
+import org.apache.storm.generated.ExecutorStats;
+
+public class ExecutorBeat {
+ private final int timeSecs;
+ private final int uptime;
+ private final ExecutorStats stats;
+
+ public ExecutorBeat(int timeSecs, int uptime, ExecutorStats stats) {
+ this.timeSecs = timeSecs;
+ this.uptime = uptime;
+ this.stats = stats;
+ }
+
+ public int getTimeSecs() {
+ return timeSecs;
+ }
+
+ public int getUptime() {
+ return uptime;
+ }
+
+ public ExecutorStats getStats() {
+ return stats;
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/jvm/org/apache/storm/cluster/IStateStorage.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/IStateStorage.java b/storm-core/src/jvm/org/apache/storm/cluster/IStateStorage.java
index 1a2b14f..0b6f043 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/IStateStorage.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/IStateStorage.java
@@ -17,8 +17,8 @@
*/
package org.apache.storm.cluster;
-import clojure.lang.APersistentMap;
import java.util.List;
+import java.util.Map;
import org.apache.curator.framework.state.ConnectionStateListener;
import org.apache.storm.callback.ZKStateChangedCallback;
@@ -42,6 +42,9 @@ import org.apache.zookeeper.data.ACL;
*/
public interface IStateStorage {
+ public static final String DATA = "data";
+ public static final String VERSION = "version";
+
/**
* Registers a callback function that gets called when CuratorEvents happen.
* @param callback is a clojure IFn that accepts the type - translated to
@@ -149,14 +152,14 @@ public interface IStateStorage {
/**
* Get the data at the node along with its version. Data is returned
- * in an APersistentMap with clojure keyword keys :data and :version.
+ * in an Map with the keys data and version.
* @param path The path to look under
* @param watch Whether or not to set a watch on the path. Watched paths
* emit events which are consumed by functions registered with the
* register method. Very useful for catching updates to nodes.
- * @return An APersistentMap in the form {:data data :version version}
+ * @return An Map in the form {:data data :version version}
*/
- APersistentMap get_data_with_version(String path, boolean watch);
+ Map get_data_with_version(String path, boolean watch);
/**
* Write a worker heartbeat at the path.
http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
index 01cf56a..c88935e 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
@@ -17,8 +17,6 @@
*/
package org.apache.storm.cluster;
-import clojure.lang.APersistentMap;
-import clojure.lang.IFn;
import org.apache.storm.generated.*;
import org.apache.storm.nimbus.NimbusInfo;
@@ -27,13 +25,13 @@ import java.util.List;
import java.util.Map;
public interface IStormClusterState {
- public List<String> assignments(IFn callback);
+ public List<String> assignments(Runnable callback);
- public Assignment assignmentInfo(String stormId, IFn callback);
+ public Assignment assignmentInfo(String stormId, Runnable callback);
- public APersistentMap assignmentInfoWithVersion(String stormId, IFn callback);
+ public Map assignmentInfoWithVersion(String stormId, Runnable callback);
- public Integer assignmentVersion(String stormId, IFn callback) throws Exception;
+ public Integer assignmentVersion(String stormId, Runnable callback) throws Exception;
public List<String> blobstoreInfo(String blobKey);
@@ -43,7 +41,7 @@ public interface IStormClusterState {
public List<String> activeStorms();
- public StormBase stormBase(String stormId, IFn callback);
+ public StormBase stormBase(String stormId, Runnable callback);
public ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, String node, Long port);
@@ -55,9 +53,9 @@ public interface IStormClusterState {
public void deleteTopologyProfileRequests(String stormId, ProfileRequest profileRequest);
- public Map<ExecutorInfo, APersistentMap> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort);
+ public Map<ExecutorInfo, ExecutorBeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort);
- public List<String> supervisors(IFn callback);
+ public List<String> supervisors(Runnable callback);
public SupervisorInfo supervisorInfo(String supervisorId); // returns nil if doesn't exist
@@ -73,7 +71,7 @@ public interface IStormClusterState {
public void setTopologyLogConfig(String stormId, LogConfig logConfig);
- public LogConfig topologyLogConfig(String stormId, IFn cb);
+ public LogConfig topologyLogConfig(String stormId, Runnable cb);
public void workerHeartbeat(String stormId, String node, Long port, ClusterWorkerHeartbeat info);
@@ -83,7 +81,7 @@ public interface IStormClusterState {
public void workerBackpressure(String stormId, String node, Long port, boolean on);
- public boolean topologyBackpressure(String stormId, IFn callback);
+ public boolean topologyBackpressure(String stormId, Runnable callback);
public void setupBackpressure(String stormId);
@@ -101,7 +99,7 @@ public interface IStormClusterState {
public List<String> activeKeys();
- public List<String> blobstore(IFn callback);
+ public List<String> blobstore(Runnable callback);
public void removeStorm(String stormId);
@@ -117,7 +115,7 @@ public interface IStormClusterState {
public void setCredentials(String stormId, Credentials creds, Map topoConf) throws NoSuchAlgorithmException;
- public Credentials credentials(String stormId, IFn callback);
+ public Credentials credentials(String stormId, Runnable callback);
public void disconnect();
http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java b/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
index a9c4d89..c29078e 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
@@ -17,7 +17,6 @@
*/
package org.apache.storm.cluster;
-import clojure.lang.APersistentMap;
import org.apache.curator.framework.state.ConnectionStateListener;
import org.apache.storm.callback.ZKStateChangedCallback;
import org.apache.storm.generated.*;
@@ -28,6 +27,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.List;
+import java.util.Map;
public class PaceMakerStateStorage implements IStateStorage {
@@ -104,7 +104,7 @@ public class PaceMakerStateStorage implements IStateStorage {
}
@Override
- public APersistentMap get_data_with_version(String path, boolean watch) {
+ public Map get_data_with_version(String path, boolean watch) {
return stateStorage.get_data_with_version(path, watch);
}
http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java b/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java
index eafd2e7..3111e04 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java
@@ -17,12 +17,12 @@
*/
package org.apache.storm.cluster;
-import clojure.lang.APersistentMap;
import org.apache.storm.pacemaker.PacemakerClient;
import org.apache.storm.utils.Utils;
import org.apache.zookeeper.data.ACL;
import java.util.List;
+import java.util.Map;
public class PaceMakerStateStorageFactory implements StateStorageFactory {
@@ -38,7 +38,7 @@ public class PaceMakerStateStorageFactory implements StateStorageFactory {
}
@Override
- public IStateStorage mkStore(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) {
+ public IStateStorage mkStore(Map config, Map auth_conf, List<ACL> acls, ClusterStateContext context) {
try {
return new PaceMakerStateStorage(initMakeClient(config), initZKstate(config, auth_conf, acls, context));
} catch (Exception e) {
@@ -46,19 +46,19 @@ public class PaceMakerStateStorageFactory implements StateStorageFactory {
}
}
- public static IStateStorage initZKstate(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
+ public static IStateStorage initZKstate(Map config, Map auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
return _instance.initZKstateImpl(config, auth_conf, acls, context);
}
- public static PacemakerClient initMakeClient(APersistentMap config) {
+ public static PacemakerClient initMakeClient(Map config) {
return _instance.initMakeClientImpl(config);
}
- public IStateStorage initZKstateImpl(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
+ public IStateStorage initZKstateImpl(Map config, Map auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
return ClusterUtils.mkStateStorage(config, auth_conf, acls, context);
}
- public PacemakerClient initMakeClientImpl(APersistentMap config) {
+ public PacemakerClient initMakeClientImpl(Map config) {
return new PacemakerClient(config);
}
}
http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java b/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
index 110da41..0929750 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
@@ -17,12 +17,12 @@
*/
package org.apache.storm.cluster;
-import clojure.lang.APersistentMap;
import java.util.List;
+import java.util.Map;
+
import org.apache.zookeeper.data.ACL;
public interface StateStorageFactory {
- IStateStorage mkStore(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context);
-
+ IStateStorage mkStore(Map config, Map auth_conf, List<ACL> acls, ClusterStateContext context);
}
http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
index 17c8641..5fa586a 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
@@ -34,8 +34,6 @@ import org.apache.zookeeper.data.ACL;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.PrintWriter;
-import java.io.StringWriter;
import java.security.NoSuchAlgorithmException;
import java.util.*;
import java.util.concurrent.ConcurrentHashMap;
@@ -47,17 +45,17 @@ public class StormClusterStateImpl implements IStormClusterState {
private IStateStorage stateStorage;
- private ConcurrentHashMap<String, IFn> assignmentInfoCallback;
- private ConcurrentHashMap<String, IFn> assignmentInfoWithVersionCallback;
- private ConcurrentHashMap<String, IFn> assignmentVersionCallback;
- private AtomicReference<IFn> supervisorsCallback;
+ private ConcurrentHashMap<String, Runnable> assignmentInfoCallback;
+ private ConcurrentHashMap<String, Runnable> assignmentInfoWithVersionCallback;
+ private ConcurrentHashMap<String, Runnable> assignmentVersionCallback;
+ private AtomicReference<Runnable> 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 ConcurrentHashMap<String, Runnable> backPressureCallback;
+ private AtomicReference<Runnable> assignmentsCallback;
+ private ConcurrentHashMap<String, Runnable> stormBaseCallback;
+ private AtomicReference<Runnable> blobstoreCallback;
+ private ConcurrentHashMap<String, Runnable> credentialsCallback;
+ private ConcurrentHashMap<String, Runnable> logConfigCallback;
private List<ACL> acls;
private String stateId;
@@ -129,20 +127,20 @@ public class StormClusterStateImpl implements IStormClusterState {
}
- protected void issueCallback(AtomicReference<IFn> cb) {
- IFn callback = cb.getAndSet(null);
+ protected void issueCallback(AtomicReference<Runnable> cb) {
+ Runnable callback = cb.getAndSet(null);
if (callback != null)
- callback.invoke();
+ callback.run();
}
- protected void issueMapCallback(ConcurrentHashMap<String, IFn> callbackConcurrentHashMap, String key) {
- IFn callback = callbackConcurrentHashMap.remove(key);
+ protected void issueMapCallback(ConcurrentHashMap<String, Runnable> callbackConcurrentHashMap, String key) {
+ Runnable callback = callbackConcurrentHashMap.remove(key);
if (callback != null)
- callback.invoke();
+ callback.run();
}
@Override
- public List<String> assignments(IFn callback) {
+ public List<String> assignments(Runnable callback) {
if (callback != null) {
assignmentsCallback.set(callback);
}
@@ -150,7 +148,7 @@ public class StormClusterStateImpl implements IStormClusterState {
}
@Override
- public Assignment assignmentInfo(String stormId, IFn callback) {
+ public Assignment assignmentInfo(String stormId, Runnable callback) {
if (callback != null) {
assignmentInfoCallback.put(stormId, callback);
}
@@ -159,23 +157,25 @@ public class StormClusterStateImpl implements IStormClusterState {
}
@Override
- public APersistentMap assignmentInfoWithVersion(String stormId, IFn callback) {
+ public Map assignmentInfoWithVersion(String stormId, Runnable callback) {
+ Map map = new HashMap();
if (callback != null) {
assignmentInfoWithVersionCallback.put(stormId, callback);
}
Assignment assignment = null;
Integer version = 0;
- APersistentMap aPersistentMap = stateStorage.get_data_with_version(ClusterUtils.assignmentPath(stormId), callback != null);
- if (aPersistentMap != null) {
- assignment = ClusterUtils.maybeDeserialize((byte[]) aPersistentMap.get(RT.keyword(null, "data")), Assignment.class);
- version = (Integer) aPersistentMap.get(RT.keyword(null, "version"));
+ Map dataWithVersionMap = stateStorage.get_data_with_version(ClusterUtils.assignmentPath(stormId), callback != null);
+ if (dataWithVersionMap != null) {
+ assignment = ClusterUtils.maybeDeserialize((byte[]) dataWithVersionMap.get(IStateStorage.DATA), Assignment.class);
+ version = (Integer) dataWithVersionMap.get(IStateStorage.VERSION);
}
- APersistentMap map = new PersistentArrayMap(new Object[] { RT.keyword(null, "data"), assignment, RT.keyword(null, "version"), version });
+ map.put(IStateStorage.DATA, assignment);
+ map.put(IStateStorage.VERSION, version);
return map;
}
@Override
- public Integer assignmentVersion(String stormId, IFn callback) throws Exception {
+ public Integer assignmentVersion(String stormId, Runnable callback) throws Exception {
if (callback != null) {
assignmentVersionCallback.put(stormId, callback);
}
@@ -227,7 +227,7 @@ public class StormClusterStateImpl implements IStormClusterState {
}
@Override
- public StormBase stormBase(String stormId, IFn callback) {
+ public StormBase stormBase(String stormId, Runnable callback) {
if (callback != null) {
stormBaseCallback.put(stormId, callback);
}
@@ -298,10 +298,10 @@ public class StormClusterStateImpl implements IStormClusterState {
* @return
*/
@Override
- public Map<ExecutorInfo, APersistentMap> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort) {
- Map<ExecutorInfo, APersistentMap> executorWhbs = new HashMap<>();
+ public Map<ExecutorInfo, ExecutorBeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort) {
+ Map<ExecutorInfo, ExecutorBeat> executorWhbs = new HashMap<>();
- Map<NodeInfo, List<List<Long>>> nodePortExecutors = ClusterUtils.reverseMap(executorNodePort);
+ Map<NodeInfo, List<List<Long>>> nodePortExecutors = Utils.reverseMap(executorNodePort);
for (Map.Entry<NodeInfo, List<List<Long>>> entry : nodePortExecutors.entrySet()) {
@@ -319,7 +319,7 @@ public class StormClusterStateImpl implements IStormClusterState {
}
@Override
- public List<String> supervisors(IFn callback) {
+ public List<String> supervisors(Runnable callback) {
if (callback != null) {
supervisorsCallback.set(callback);
}
@@ -342,7 +342,7 @@ public class StormClusterStateImpl implements IStormClusterState {
try {
stateStorage.delete_worker_hb(ClusterUtils.workerbeatStormRoot(stormId));
} catch (Exception e) {
- if (Zookeeper.exceptionCause(KeeperException.class, e)) {
+ if (Utils.exceptionCauseIsInstanceOf(KeeperException.class, e)) {
// do nothing
LOG.warn("Could not teardown heartbeats for {}.", stormId);
} else {
@@ -356,7 +356,7 @@ public class StormClusterStateImpl implements IStormClusterState {
try {
stateStorage.delete_node(ClusterUtils.errorStormRoot(stormId));
} catch (Exception e) {
- if (Zookeeper.exceptionCause(KeeperException.class, e)) {
+ if (Utils.exceptionCauseIsInstanceOf(KeeperException.class, e)) {
// do nothing
LOG.warn("Could not teardown errors for {}.", stormId);
} else {
@@ -381,7 +381,7 @@ public class StormClusterStateImpl implements IStormClusterState {
}
@Override
- public LogConfig topologyLogConfig(String stormId, IFn cb) {
+ public LogConfig topologyLogConfig(String stormId, Runnable cb) {
String path = ClusterUtils.logConfigPath(stormId);
return ClusterUtils.maybeDeserialize(stateStorage.get_data(path, cb != null), LogConfig.class);
}
@@ -437,7 +437,7 @@ public class StormClusterStateImpl implements IStormClusterState {
* @return
*/
@Override
- public boolean topologyBackpressure(String stormId, IFn callback) {
+ public boolean topologyBackpressure(String stormId, Runnable callback) {
if (callback != null) {
backPressureCallback.put(stormId, callback);
}
@@ -568,7 +568,7 @@ public class StormClusterStateImpl implements IStormClusterState {
// blobstore state
@Override
- public List<String> blobstore(IFn callback) {
+ public List<String> blobstore(Runnable callback) {
if (callback != null) {
blobstoreCallback.set(callback);
}
@@ -602,7 +602,7 @@ public class StormClusterStateImpl implements IStormClusterState {
String path = ClusterUtils.errorPath(stormId, componentId);
String lastErrorPath = ClusterUtils.lastErrorPath(stormId, componentId);
- ErrorInfo errorInfo = new ErrorInfo(ClusterUtils.StringifyError(error), Time.currentTimeSecs());
+ ErrorInfo errorInfo = new ErrorInfo(ClusterUtils.stringifyError(error), Time.currentTimeSecs());
errorInfo.set_host(node);
errorInfo.set_port(port.intValue());
byte[] serData = Utils.serialize(errorInfo);
@@ -669,7 +669,7 @@ public class StormClusterStateImpl implements IStormClusterState {
}
@Override
- public Credentials credentials(String stormId, IFn callback) {
+ public Credentials credentials(String stormId, Runnable callback) {
if (callback != null) {
credentialsCallback.put(stormId, callback);
}
http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
index b277751..56115ce 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
@@ -17,7 +17,6 @@
*/
package org.apache.storm.cluster;
-import clojure.lang.APersistentMap;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.state.*;
import org.apache.curator.framework.state.ConnectionState;
@@ -220,7 +219,7 @@ public class ZKStateStorage implements IStateStorage {
}
@Override
- public APersistentMap get_data_with_version(String path, boolean watch) {
+ public Map get_data_with_version(String path, boolean watch) {
return Zookeeper.getDataWithVersion(zkReader, path, watch);
}
http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
index 956c20e..232488b 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
@@ -17,16 +17,16 @@
*/
package org.apache.storm.cluster;
-import clojure.lang.APersistentMap;
import org.apache.storm.utils.Utils;
import org.apache.zookeeper.data.ACL;
import java.util.List;
+import java.util.Map;
public class ZKStateStorageFactory implements StateStorageFactory {
@Override
- public IStateStorage mkStore(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) {
+ public IStateStorage mkStore(Map config, Map auth_conf, List<ACL> acls, ClusterStateContext context) {
try {
return new ZKStateStorage(config, auth_conf, acls, context);
} catch (Exception e) {
http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java b/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
index f80b0a4..e5b2666 100644
--- a/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
+++ b/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
@@ -17,15 +17,11 @@
*/
package org.apache.storm.zookeeper;
-import clojure.lang.APersistentMap;
-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.api.CuratorEvent;
import org.apache.curator.framework.api.CuratorEventType;
import org.apache.curator.framework.api.CuratorListener;
-import org.apache.curator.framework.api.UnhandledErrorListener;
import org.apache.curator.framework.recipes.leader.LeaderLatch;
import org.apache.curator.framework.recipes.leader.LeaderLatchListener;
import org.apache.curator.framework.recipes.leader.Participant;
@@ -33,6 +29,7 @@ import org.apache.curator.framework.state.ConnectionStateListener;
import org.apache.storm.Config;
import org.apache.storm.callback.DefaultWatcherCallBack;
import org.apache.storm.callback.WatcherCallBack;
+import org.apache.storm.cluster.IStateStorage;
import org.apache.storm.nimbus.ILeaderElector;
import org.apache.storm.nimbus.NimbusInfo;
import org.apache.storm.utils.Utils;
@@ -47,17 +44,13 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
-import java.io.IOException;
import java.net.BindException;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
-import java.util.Arrays;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
import java.util.concurrent.atomic.AtomicReference;
-import java.util.Vector;
+
public class Zookeeper {
private static Logger LOG = LoggerFactory.getLogger(Zookeeper.class);
@@ -169,7 +162,7 @@ public class Zookeeper {
zk.delete().deletingChildrenIfNeeded().forPath(normalizePath(path));
}
} catch (Exception e) {
- if (exceptionCause(KeeperException.NodeExistsException.class, e)) {
+ if (Utils.exceptionCauseIsInstanceOf(KeeperException.NodeExistsException.class, e)) {
// do nothing
LOG.info("delete {} failed.", path, e);
} else {
@@ -195,7 +188,7 @@ public class Zookeeper {
try {
createNode(zk, npath, byteArray, org.apache.zookeeper.CreateMode.PERSISTENT, acls);
} catch (Exception e) {
- if (exceptionCause(KeeperException.NodeExistsException.class, e)) {
+ if (Utils.exceptionCauseIsInstanceOf(KeeperException.NodeExistsException.class, e)) {
// this can happen when multiple clients doing mkdir at same time
}
}
@@ -224,7 +217,7 @@ public class Zookeeper {
}
}
} catch (Exception e) {
- if (exceptionCause(KeeperException.NoNodeException.class, e)) {
+ if (Utils.exceptionCauseIsInstanceOf(KeeperException.NoNodeException.class, e)) {
// this is fine b/c we still have a watch from the successful exists call
} else {
throw Utils.wrapInRuntime(e);
@@ -312,7 +305,7 @@ public class Zookeeper {
}
LOG.info("Starting inprocess zookeeper at port {} and dir {}", report, localdir);
factory.startup(zk);
- return Arrays.asList((Object)new Long(report), (Object)factory);
+ return Arrays.asList((Object) new Long(report), (Object) factory);
}
public static void shutdownInprocessZookeeper(NIOServerCnxnFactory handle) {
@@ -361,9 +354,8 @@ public class Zookeeper {
return new LeaderElectorImp(conf, servers, zk, leaderLockPath, id, leaderLatchAtomicReference, leaderLatchListenerAtomicReference);
}
- // To update @return to be a Map
- public static APersistentMap getDataWithVersion(CuratorFramework zk, String path, boolean watch) {
- APersistentMap map = null;
+ public static Map getDataWithVersion(CuratorFramework zk, String path, boolean watch) {
+ Map map = new HashMap();
try {
byte[] bytes = null;
Stat stats = new Stat();
@@ -376,11 +368,12 @@ public class Zookeeper {
}
if (bytes != null) {
int version = stats.getVersion();
- map = new PersistentArrayMap(new Object[] { RT.keyword(null, "data"), bytes, RT.keyword(null, "version"), version });
+ map.put(IStateStorage.DATA, bytes);
+ map.put(IStateStorage.VERSION, version);
}
}
} catch (Exception e) {
- if (exceptionCause(KeeperException.NoNodeException.class, e)) {
+ if (Utils.exceptionCauseIsInstanceOf(KeeperException.NoNodeException.class, e)) {
// this is fine b/c we still have a watch from the successful exists call
} else {
Utils.wrapInRuntime(e);
@@ -423,19 +416,4 @@ public class Zookeeper {
String rtn = toksToPath(tokenizePath(path));
return rtn;
}
-
- // To remove exceptionCause if port Utils.try-cause to java
- public static boolean exceptionCause(Class klass, Throwable t) {
- boolean ret = false;
- Throwable throwable = t;
- while (throwable != null) {
- if (throwable.getClass() == klass) {
- ret = true;
- break;
- }
- throwable = throwable.getCause();
- }
- return ret;
- }
-
}
[23/27] storm git commit: convert ExecutorStats to stats of clojure
by clojurify-executor-stats
Posted by bo...@apache.org.
convert ExecutorStats to stats of clojure by clojurify-executor-stats
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/d7095251
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/d7095251
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/d7095251
Branch: refs/heads/master
Commit: d70952513336532c80957e5398bfbedacff78cb7
Parents: 581cd29
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Tue Feb 23 22:05:50 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Tue Feb 23 22:05:50 2016 +0800
----------------------------------------------------------------------
storm-core/src/clj/org/apache/storm/converter.clj | 2 +-
storm-core/src/clj/org/apache/storm/daemon/nimbus.clj | 9 ++++-----
2 files changed, 5 insertions(+), 6 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/d7095251/storm-core/src/clj/org/apache/storm/converter.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/converter.clj b/storm-core/src/clj/org/apache/storm/converter.clj
index c845cd4..e269c5d 100644
--- a/storm-core/src/clj/org/apache/storm/converter.clj
+++ b/storm-core/src/clj/org/apache/storm/converter.clj
@@ -241,7 +241,7 @@
(defn clojurify-zk-executor-hb [^ExecutorBeat executor-hb]
(if executor-hb
- {:stats (.getStats executor-hb)
+ {:stats (clojurify-executor-stats (.getStats executor-hb))
:uptime (.getUptime executor-hb)
:time-secs (.getTimeSecs executor-hb)
}
http://git-wip-us.apache.org/repos/asf/storm/blob/d7095251/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index e43bab9..2f6587a 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@ -597,7 +597,6 @@
(->> (dofor [[^ExecutorInfo executor-info ^ExecutorBeat executor-heartbeat] executor-stats-clojurify]
{[(.get_task_start executor-info) (.get_task_end executor-info)] (clojurify-zk-executor-hb executor-heartbeat)})
(apply merge)))
-
cache (update-heartbeat-cache (@(:heartbeats-cache nimbus) storm-id)
executor-beats
all-executors
@@ -1918,16 +1917,16 @@
executor-summaries (dofor [[executor [node port]] (:executor->node+port assignment)]
(let [host (-> assignment :node->host (get node))
heartbeat (get beats executor)
- stats (:stats heartbeat)
- stats (if stats
- (stats/thriftify-executor-stats stats))]
+ excutorstats (:stats heartbeat)
+ excutorstats (if excutorstats
+ (stats/thriftify-executor-stats excutorstats))]
(doto
(ExecutorSummary. (thriftify-executor-id executor)
(-> executor first task->component)
host
port
(Utils/nullToZero (:uptime heartbeat)))
- (.set_stats stats))
+ (.set_stats excutorstats))
))
topo-info (TopologyInfo. storm-id
storm-name
[16/27] storm git commit: fix nimbus_test.clj
Posted by bo...@apache.org.
fix nimbus_test.clj
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/e0f3cb5f
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/e0f3cb5f
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/e0f3cb5f
Branch: refs/heads/master
Commit: e0f3cb5f322c1dc09d05803e93d1fb3e6a3baff0
Parents: 419be35
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Tue Feb 16 11:40:48 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Tue Feb 16 11:40:48 2016 +0800
----------------------------------------------------------------------
storm-core/test/clj/org/apache/storm/nimbus_test.clj | 2 --
1 file changed, 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/e0f3cb5f/storm-core/test/clj/org/apache/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/nimbus_test.clj b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
index 2a65efc..09c4371 100644
--- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj
+++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
@@ -1411,8 +1411,6 @@
cluster-utils (Mockito/mock ClusterUtils)]
(with-open [_ (ConfigUtilsInstaller. fake-cu)
_ (UtilsInstaller. fake-utils)
- _ (proxy [ConfigUtils] []
- (nimbusTopoHistoryStateImpl [conf] nil))
zk-le (MockedZookeeper. (proxy [Zookeeper] []
(zkLeaderElectorImpl [conf] nil)))
mocked-cluster (MockedCluster. cluster-utils)]
[06/27] storm git commit: Merge branch 'master' into cluster
Posted by bo...@apache.org.
Merge branch 'master' into cluster
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/2a17c455
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/2a17c455
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/2a17c455
Branch: refs/heads/master
Commit: 2a17c45591692b61df0be922fdec9d10a108de27
Parents: 5916b0b d236036
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Wed Feb 3 21:18:13 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Wed Feb 3 21:18:13 2016 +0800
----------------------------------------------------------------------
CHANGELOG.md | 1 +
README.markdown | 1 +
external/storm-hdfs/README.md | 15 +++-
external/storm-hdfs/pom.xml | 22 ++++++
.../storm/hdfs/avro/AbstractAvroSerializer.java | 80 +++++++++++++++++++
.../storm/hdfs/avro/AvroSchemaRegistry.java | 28 +++++++
.../org/apache/storm/hdfs/avro/AvroUtils.java | 44 +++++++++++
.../hdfs/avro/ConfluentAvroSerializer.java | 83 ++++++++++++++++++++
.../storm/hdfs/avro/FixedAvroSerializer.java | 67 ++++++++++++++++
.../storm/hdfs/avro/GenericAvroSerializer.java | 36 +++++++++
.../storm/hdfs/bolt/AvroGenericRecordBolt.java | 4 -
.../hdfs/avro/TestFixedAvroSerializer.java | 76 ++++++++++++++++++
.../hdfs/avro/TestGenericAvroSerializer.java | 68 ++++++++++++++++
.../test/resources/FixedAvroSerializer.config | 2 +
pom.xml | 3 +
.../clj/org/apache/storm/daemon/executor.clj | 2 +-
.../src/clj/org/apache/storm/daemon/task.clj | 2 +
17 files changed, 527 insertions(+), 7 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/2a17c455/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
[02/27] storm git commit: delete zookeeper.clj
zookeeper_state_factory.clj cluster.clj, but some tests still can't pass
Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/test/clj/org/apache/storm/supervisor_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/supervisor_test.clj b/storm-core/test/clj/org/apache/storm/supervisor_test.clj
index edb161b..c98a68b 100644
--- a/storm-core/test/clj/org/apache/storm/supervisor_test.clj
+++ b/storm-core/test/clj/org/apache/storm/supervisor_test.clj
@@ -23,15 +23,18 @@
(:import [org.apache.storm.scheduler ISupervisor])
(:import [org.apache.storm.utils ConfigUtils])
(:import [org.apache.storm.generated RebalanceOptions])
- (:import [org.apache.storm.testing.staticmocking MockedConfigUtils])
+ (:import [org.apache.storm.testing.staticmocking MockedConfigUtils MockedCluster])
(:import [java.util UUID])
+ (:import [org.mockito Mockito])
+ (:import [org.mockito.exceptions.base MockitoAssertionError])
(:import [java.io File])
(:import [java.nio.file Files])
+ (:import [org.apache.storm.cluster StormZkClusterState Cluster ClusterStateContext])
(:import [java.nio.file.attribute FileAttribute])
- (:use [org.apache.storm config testing util timer log])
+ (:use [org.apache.storm config testing util timer log converter])
(:use [org.apache.storm.daemon common])
(:require [org.apache.storm.daemon [worker :as worker] [supervisor :as supervisor]]
- [org.apache.storm [thrift :as thrift] [cluster :as cluster]])
+ [org.apache.storm [thrift :as thrift]])
(:use [conjure core])
(:require [clojure.java.io :as io]))
@@ -40,7 +43,7 @@
[cluster supervisor-id port]
(let [state (:storm-cluster-state cluster)
slot-assigns (for [storm-id (.assignments state nil)]
- (let [executors (-> (.assignment-info state storm-id nil)
+ (let [executors (-> (clojurify-assignment (.assignmentInfo state storm-id nil))
:executor->node+port
reverse-map
(get [supervisor-id port] ))]
@@ -225,7 +228,7 @@
)))
(defn get-heartbeat [cluster supervisor-id]
- (.supervisor-info (:storm-cluster-state cluster) supervisor-id))
+ (clojurify-supervisor-info (.supervisorInfo (:storm-cluster-state cluster) supervisor-id)))
(defn check-heartbeat [cluster supervisor-id within-secs]
(let [hb (get-heartbeat cluster supervisor-id)
@@ -561,18 +564,22 @@
expected-acls supervisor/SUPERVISOR-ZK-ACLS
fake-isupervisor (reify ISupervisor
(getSupervisorId [this] nil)
- (getAssignmentId [this] nil))]
+ (getAssignmentId [this] nil))
+ storm-zk (Mockito/mock Cluster)]
(with-open [_ (proxy [MockedConfigUtils] []
(supervisorStateImpl [conf] nil)
- (supervisorLocalDirImpl [conf] nil))]
+ (supervisorLocalDirImpl [conf] nil))
+ storm-zk-le (MockedCluster. storm-zk)]
(stubbing [uptime-computer nil
- cluster/mk-storm-cluster-state nil
+ ; cluster/mk-storm-cluster-state nil
local-hostname nil
mk-timer 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]
- expected-acls))))))
+ (.mkStormClusterStateImpl (Mockito/verify storm-zk (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
+ ; (verify-call-times-for cluster/mk-storm-cluster-state 1)
+ ; (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
+ ; expected-acls)
+ )))))
(deftest test-write-log-metadata
(testing "supervisor writes correct data to logs metadata file"
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/test/jvm/org/apache/storm/ClusterTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/ClusterTest.java b/storm-core/test/jvm/org/apache/storm/ClusterTest.java
new file mode 100644
index 0000000..ef43afe
--- /dev/null
+++ b/storm-core/test/jvm/org/apache/storm/ClusterTest.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 org.apache.storm;
+
+
+public class ClusterTest {
+}
[10/27] storm git commit: port pacemaker_state_factory.clj
Posted by bo...@apache.org.
port pacemaker_state_factory.clj
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/2ee8bec8
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/2ee8bec8
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/2ee8bec8
Branch: refs/heads/master
Commit: 2ee8bec8458b02bca6af757ce0f1052a16c660b8
Parents: 55b86ca
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Mon Feb 15 09:37:32 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Mon Feb 15 09:37:32 2016 +0800
----------------------------------------------------------------------
.../clj/org/apache/storm/command/heartbeats.clj | 2 +-
.../src/clj/org/apache/storm/daemon/worker.clj | 2 +-
.../storm/pacemaker/pacemaker_state_factory.clj | 6 +-
storm-core/src/clj/org/apache/storm/testing.clj | 2 +-
.../org/apache/storm/cluster/ClusterUtils.java | 20 +-
.../org/apache/storm/cluster/IStateStorage.java | 219 ++++++++++++++++++
.../storm/cluster/IStormClusterState.java | 129 +++++++++++
.../storm/cluster/PaceMakerStateStorage.java | 212 ++++++++++++++++++
.../cluster/PaceMakerStateStorageFactory.java | 64 ++++++
.../org/apache/storm/cluster/StateStorage.java | 220 -------------------
.../storm/cluster/StateStorageFactory.java | 2 +-
.../apache/storm/cluster/StormClusterState.java | 129 -----------
.../storm/cluster/StormClusterStateImpl.java | 8 +-
.../apache/storm/cluster/ZKStateStorage.java | 4 +-
.../storm/cluster/ZKStateStorageFactory.java | 2 +-
.../apache/storm/pacemaker/PacemakerClient.java | 1 -
.../MockedPaceMakerStateStorageFactory.java | 32 +++
.../test/clj/org/apache/storm/cluster_test.clj | 10 +-
.../storm/pacemaker_state_factory_test.clj | 57 ++---
19 files changed, 715 insertions(+), 406 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/command/heartbeats.clj b/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
index af86b69..c4413f0 100644
--- a/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
+++ b/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
@@ -27,7 +27,7 @@
(defn -main [command path & args]
(let [conf (clojurify-structure (ConfigUtils/readStormConfig))
- cluster (ClusterUtils/mkDistributedClusterState conf conf nil (ClusterStateContext.))]
+ cluster (ClusterUtils/mkStateStorage conf conf nil (ClusterStateContext.))]
(println "Command: [" command "]")
(condp = command
"list"
http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/clj/org/apache/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/worker.clj b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
index a793009..ae5be57 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
@@ -596,7 +596,7 @@
(let [storm-conf (ConfigUtils/readSupervisorStormConf conf storm-id)
storm-conf (clojurify-structure (ConfigUtils/overrideLoginConfigWithSystemProperty storm-conf))
acls (Utils/getWorkerACL storm-conf)
- state-store (ClusterUtils/mkDistributedClusterState conf storm-conf acls (ClusterStateContext. DaemonType/WORKER))
+ state-store (ClusterUtils/mkStateStorage conf storm-conf acls (ClusterStateContext. DaemonType/WORKER))
storm-cluster-state (ClusterUtils/mkStormClusterState state-store acls (ClusterStateContext.))
initial-credentials (clojurify-crdentials (.credentials storm-cluster-state storm-id nil))
auto-creds (AuthUtils/GetAutoCredentials storm-conf)
http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj b/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
index 28f792d..a36da3a 100644
--- a/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
+++ b/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
@@ -23,7 +23,7 @@
(:import [org.apache.storm.generated
HBExecutionException HBServerMessageType HBMessage
HBMessageData HBPulse]
- [org.apache.storm.cluster ZKStateStorage StateStorage ClusterUtils]
+ [org.apache.storm.cluster ZKStateStorage ClusterUtils IStateStorage]
[org.apache.storm.pacemaker PacemakerClient])
(:gen-class
:implements [org.apache.storm.cluster.StateStorageFactory]))
@@ -33,7 +33,7 @@
(PacemakerClient. conf))
(defn makeZKState [conf auth-conf acls context]
- (ClusterUtils/mkDistributedClusterState conf auth-conf acls context))
+ (ClusterUtils/mkStateStorage conf auth-conf acls context))
(def max-retries 10)
@@ -42,7 +42,7 @@
pacemaker-client (makeClient conf)]
(reify
- StateStorage
+ IStateStorage
;; Let these pass through to the zk-state. We only want to handle heartbeats.
(register [this callback] (.register zk-state callback))
(unregister [this callback] (.unregister zk-state callback))
http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/clj/org/apache/storm/testing.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/testing.clj b/storm-core/src/clj/org/apache/storm/testing.clj
index 470a14f..5a0bdf2 100644
--- a/storm-core/src/clj/org/apache/storm/testing.clj
+++ b/storm-core/src/clj/org/apache/storm/testing.clj
@@ -158,7 +158,7 @@
:port-counter port-counter
:daemon-conf daemon-conf
:supervisors (atom [])
- :state (ClusterUtils/mkDistributedClusterState daemon-conf nil nil (ClusterStateContext.))
+ :state (ClusterUtils/mkStateStorage daemon-conf nil nil (ClusterStateContext.))
:storm-cluster-state (ClusterUtils/mkStormClusterState daemon-conf nil (ClusterStateContext.))
:tmp-dirs (atom [nimbus-tmp zk-tmp])
:zookeeper (if (not-nil? zk-handle) zk-handle)
http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java b/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
index 9fd36ca..b30d1d2 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
@@ -194,20 +194,20 @@ public class ClusterUtils {
return executorWhb;
}
- public StormClusterState mkStormClusterStateImpl(Object StateStorage, List<ACL> acls, ClusterStateContext context) throws Exception {
- if (StateStorage instanceof StateStorage) {
- return new StormClusterStateImpl((StateStorage) StateStorage, acls, context, false);
+ public IStormClusterState mkStormClusterStateImpl(Object stateStorage, List<ACL> acls, ClusterStateContext context) throws Exception {
+ if (stateStorage instanceof IStateStorage) {
+ return new StormClusterStateImpl((IStateStorage) stateStorage, acls, context, false);
} else {
- StateStorage Storage = _instance.mkDistributedClusterStateImpl((APersistentMap) StateStorage, (APersistentMap) StateStorage, acls, context);
+ IStateStorage Storage = _instance.mkStateStorageImpl((APersistentMap) stateStorage, (APersistentMap) stateStorage, acls, context);
return new StormClusterStateImpl(Storage, acls, context, true);
}
}
- public StateStorage mkDistributedClusterStateImpl(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context)
+ public IStateStorage mkStateStorageImpl(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context)
throws Exception {
String className = null;
- StateStorage stateStorage = null;
+ IStateStorage stateStorage = null;
if (config.get(Config.STORM_CLUSTER_STATE_STORE) != null) {
className = (String) config.get(Config.STORM_CLUSTER_STATE_STORE);
} else {
@@ -215,16 +215,16 @@ public class ClusterUtils {
}
Class clazz = Class.forName(className);
StateStorageFactory storageFactory = (StateStorageFactory) clazz.newInstance();
- stateStorage = storageFactory.mkState(config, auth_conf, acls, context);
+ stateStorage = storageFactory.mkStore(config, auth_conf, acls, context);
return stateStorage;
}
- public static StateStorage mkDistributedClusterState(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context)
+ public static IStateStorage mkStateStorage(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context)
throws Exception {
- return _instance.mkDistributedClusterStateImpl(config, auth_conf, acls, context);
+ return _instance.mkStateStorageImpl(config, auth_conf, acls, context);
}
- public static StormClusterState mkStormClusterState(Object StateStorage, List<ACL> acls, ClusterStateContext context) throws Exception {
+ public static IStormClusterState mkStormClusterState(Object StateStorage, List<ACL> acls, ClusterStateContext context) throws Exception {
return _instance.mkStormClusterStateImpl(StateStorage, acls, context);
}
http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/jvm/org/apache/storm/cluster/IStateStorage.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/IStateStorage.java b/storm-core/src/jvm/org/apache/storm/cluster/IStateStorage.java
new file mode 100644
index 0000000..1a2b14f
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/IStateStorage.java
@@ -0,0 +1,219 @@
+/**
+ * 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 java.util.List;
+
+import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.storm.callback.ZKStateChangedCallback;
+import org.apache.zookeeper.data.ACL;
+
+/**
+ * StateStorage provides the API for the pluggable state store used by the
+ * Storm daemons. Data is stored in path/value format, and the store supports
+ * listing sub-paths at a given path.
+ * All data should be available across all nodes with eventual consistency.
+ *
+ * IMPORTANT NOTE:
+ * Heartbeats have different api calls used to interact with them. The root
+ * path (/) may or may not be the same as the root path for the other api calls.
+ *
+ * For example, performing these two calls:
+ * set_data("/path", data, acls);
+ * void set_worker_hb("/path", heartbeat, acls);
+ * may or may not cause a collision in "/path".
+ * Never use the same paths with the *_hb* methods as you do with the others.
+ */
+public interface IStateStorage {
+
+ /**
+ * Registers a callback function that gets called when CuratorEvents happen.
+ * @param callback is a clojure IFn that accepts the type - translated to
+ * clojure keyword as in zookeeper - and the path: (callback type path)
+ * @return is an id that can be passed to unregister(...) to unregister the
+ * callback.
+ */
+ String register(ZKStateChangedCallback callback);
+
+ /**
+ * Unregisters a callback function that was registered with register(...).
+ * @param id is the String id that was returned from register(...).
+ */
+ void unregister(String id);
+
+ /**
+ * Path will be appended with a monotonically increasing integer, a new node
+ * will be created there, and data will be put at that node.
+ * @param path The path that the monotonically increasing integer suffix will
+ * be added to.
+ * @param data The data that will be written at the suffixed path's node.
+ * @param acls The acls to apply to the path. May be null.
+ * @return The path with the integer suffix appended.
+ */
+ String create_sequential(String path, byte[] data, List<ACL> acls);
+
+ /**
+ * Creates nodes for path and all its parents. Path elements are separated by
+ * a "/", as in *nix filesystem notation. Equivalent to mkdir -p in *nix.
+ * @param path The path to create, along with all its parents.
+ * @param acls The acls to apply to the path. May be null.
+ * @return path
+ */
+ void mkdirs(String path, List<ACL> acls);
+
+ /**
+ * Deletes the node at a given path, and any child nodes that may exist.
+ * @param path The path to delete
+ */
+ void delete_node(String path);
+
+ /**
+ * Creates an ephemeral node at path. Ephemeral nodes are destroyed
+ * by the store when the client disconnects.
+ * @param path The path where a node will be created.
+ * @param data The data to be written at the node.
+ * @param acls The acls to apply to the path. May be null.
+ */
+ void set_ephemeral_node(String path, byte[] data, List<ACL> acls);
+
+ /**
+ * Gets the 'version' of the node at a path. Optionally sets a watch
+ * on that node. The version should increase whenever a write happens.
+ * @param path The path to get the version of.
+ * @param watch Whether or not to set a watch on the path. Watched paths
+ * emit events which are consumed by functions registered with the
+ * register method. Very useful for catching updates to nodes.
+ * @return The integer version of this node.
+ */
+ Integer get_version(String path, boolean watch) throws Exception;
+
+ /**
+ * Check if a node exists and optionally set a watch on the path.
+ * @param path The path to check for the existence of a node.
+ * @param watch Whether or not to set a watch on the path. Watched paths
+ * emit events which are consumed by functions registered with the
+ * register method. Very useful for catching updates to nodes.
+ * @return Whether or not a node exists at path.
+ */
+ boolean node_exists(String path, boolean watch);
+
+ /**
+ * Get a list of paths of all the child nodes which exist immediately
+ * under path.
+ * @param path The path to look under
+ * @param watch Whether or not to set a watch on the path. Watched paths
+ * emit events which are consumed by functions registered with the
+ * register method. Very useful for catching updates to nodes.
+ * @return list of string paths under path.
+ */
+ List<String> get_children(String path, boolean watch);
+
+ /**
+ * Close the connection to the data store.
+ */
+ void close();
+
+ /**
+ * Set the value of the node at path to data.
+ * @param path The path whose node we want to set.
+ * @param data The data to put in the node.
+ * @param acls The acls to apply to the path. May be null.
+ */
+ void set_data(String path, byte[] data, List<ACL> acls);
+
+ /**
+ * Get the data from the node at path
+ * @param path The path to look under
+ * @param watch Whether or not to set a watch on the path. Watched paths
+ * emit events which are consumed by functions registered with the
+ * register method. Very useful for catching updates to nodes.
+ * @return The data at the node.
+ */
+ byte[] get_data(String path, boolean watch);
+
+ /**
+ * Get the data at the node along with its version. Data is returned
+ * in an APersistentMap with clojure keyword keys :data and :version.
+ * @param path The path to look under
+ * @param watch Whether or not to set a watch on the path. Watched paths
+ * emit events which are consumed by functions registered with the
+ * register method. Very useful for catching updates to nodes.
+ * @return An APersistentMap in the form {:data data :version version}
+ */
+ APersistentMap get_data_with_version(String path, boolean watch);
+
+ /**
+ * Write a worker heartbeat at the path.
+ * @param path The path whose node we want to set.
+ * @param data The data to put in the node.
+ * @param acls The acls to apply to the path. May be null.
+ */
+ void set_worker_hb(String path, byte[] data, List<ACL> acls);
+
+ /**
+ * Get the heartbeat from the node at path
+ * @param path The path to look under
+ * @param watch Whether or not to set a watch on the path. Watched paths
+ * emit events which are consumed by functions registered with the
+ * register method. Very useful for catching updates to nodes.
+ * @return The heartbeat at the node.
+ */
+ byte[] get_worker_hb(String path, boolean watch);
+
+ /**
+ * Get a list of paths of all the child nodes which exist immediately
+ * under path. This is similar to get_children, but must be used for
+ * any nodes
+ * @param path The path to look under
+ * @param watch Whether or not to set a watch on the path. Watched paths
+ * emit events which are consumed by functions registered with the
+ * register method. Very useful for catching updates to nodes.
+ * @return list of string paths under path.
+ */
+ List<String> get_worker_hb_children(String path, boolean watch);
+
+ /**
+ * Deletes the heartbeat at a given path, and any child nodes that may exist.
+ * @param path The path to delete.
+ */
+ void delete_worker_hb(String path);
+
+ /**
+ * Add a StateStorageListener to the connection.
+ * @param listener A StateStorageListener to handle changing cluster state
+ * events.
+ */
+ void add_listener(final ConnectionStateListener listener);
+
+ /**
+ * Force consistency on a path. Any writes committed on the path before
+ * this call will be completely propagated when it returns.
+ * @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/2ee8bec8/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
new file mode 100644
index 0000000..59d1af7
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
@@ -0,0 +1,129 @@
+/**
+ * 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 org.apache.storm.generated.*;
+import org.apache.storm.nimbus.NimbusInfo;
+
+import java.security.NoSuchAlgorithmException;
+import java.util.List;
+import java.util.Map;
+
+public interface IStormClusterState {
+ public List<String> assignments(IFn callback);
+
+ public Assignment assignmentInfo(String stormId, IFn callback);
+
+ public APersistentMap assignmentInfoWithVersion(String stormId, IFn callback);
+
+ public Integer assignmentVersion(String stormId, IFn callback) throws Exception;
+
+ // returns key information under /storm/blobstore/key
+ public List<String> blobstoreInfo(String blobKey);
+
+ // returns list of nimbus summaries stored under /stormroot/nimbuses/<nimbus-ids> -> <data>
+ public List nimbuses();
+
+ // adds the NimbusSummary to /stormroot/nimbuses/nimbus-id
+ public void addNimbusHost(String nimbusId, NimbusSummary nimbusSummary);
+
+ public List<String> activeStorms();
+
+ public StormBase stormBase(String stormId, IFn callback);
+
+ public ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, String node, Long port);
+
+ public List<ProfileRequest> getWorkerProfileRequests(String stormId, NodeInfo nodeInfo, boolean isThrift);
+
+ public List<ProfileRequest> getTopologyProfileRequests(String stormId, boolean isThrift);
+
+ public void setWorkerProfileRequest(String stormId, ProfileRequest profileRequest);
+
+ public void deleteTopologyProfileRequests(String stormId, ProfileRequest profileRequest);
+
+ public Map<ExecutorInfo, ClusterWorkerHeartbeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort);
+
+ public List<String> supervisors(IFn callback);
+
+ public SupervisorInfo supervisorInfo(String supervisorId); // returns nil if doesn't exist
+
+ public void setupHeatbeats(String stormId);
+
+ public void teardownHeartbeats(String stormId);
+
+ public void teardownTopologyErrors(String stormId);
+
+ public List<String> heartbeatStorms();
+
+ public List<String> errorTopologies();
+
+ public void setTopologyLogConfig(String stormId, LogConfig logConfig);
+
+ public LogConfig topologyLogConfig(String stormId, IFn cb);
+
+ public void workerHeartbeat(String stormId, String node, Long port, ClusterWorkerHeartbeat info);
+
+ public void removeWorkerHeartbeat(String stormId, String node, Long port);
+
+ public void supervisorHeartbeat(String supervisorId, SupervisorInfo info);
+
+ public void workerBackpressure(String stormId, String node, Long port, boolean on);
+
+ public boolean topologyBackpressure(String stormId, IFn callback);
+
+ public void setupBackpressure(String stormId);
+
+ public void removeWorkerBackpressure(String stormId, String node, Long port);
+
+ public void activateStorm(String stormId, StormBase stormBase);
+
+ public void updateStorm(String stormId, StormBase newElems);
+
+ public void removeStormBase(String stormId);
+
+ public void setAssignment(String stormId, Assignment info);
+
+ // sets up information related to key consisting of nimbus
+ // host:port and version info of the blob
+ public void setupBlobstore(String key, NimbusInfo nimbusInfo, Integer versionInfo);
+
+ public List<String> activeKeys();
+
+ public List<String> blobstore(IFn callback);
+
+ public void removeStorm(String stormId);
+
+ public void removeBlobstoreKey(String blobKey);
+
+ public void removeKeyVersion(String blobKey);
+
+ public void reportError(String stormId, String componentId, String node, Long port, String error);
+
+ public List<ErrorInfo> errors(String stormId, String componentId);
+
+ public ErrorInfo lastError(String stormId, String componentId);
+
+ public void setCredentials(String stormId, Credentials creds, Map topoConf) throws NoSuchAlgorithmException;
+
+ public Credentials credentials(String stormId, IFn callback);
+
+ public void disconnect();
+
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java b/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
new file mode 100644
index 0000000..1226c55
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
@@ -0,0 +1,212 @@
+/**
+ * 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 org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.storm.callback.ZKStateChangedCallback;
+import org.apache.storm.generated.*;
+import org.apache.storm.pacemaker.PacemakerClient;
+import org.apache.storm.utils.Utils;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+public class PaceMakerStateStorage implements IStateStorage {
+
+ private static Logger LOG = LoggerFactory.getLogger(PaceMakerStateStorage.class);
+
+ private PacemakerClient pacemakerClient;
+ private IStateStorage stateStorage;
+ private static final int maxRetries = 10;
+
+ public PaceMakerStateStorage(PacemakerClient pacemakerClient, IStateStorage stateStorage) throws Exception {
+ this.pacemakerClient = pacemakerClient;
+ this.stateStorage = stateStorage;
+ }
+
+ @Override
+ public String register(ZKStateChangedCallback callback) {
+ return stateStorage.register(callback);
+ }
+
+ @Override
+ public void unregister(String id) {
+ stateStorage.unregister(id);
+ }
+
+ @Override
+ public String create_sequential(String path, byte[] data, List<ACL> acls) {
+ return stateStorage.create_sequential(path, data, acls);
+ }
+
+ @Override
+ public void mkdirs(String path, List<ACL> acls) {
+ stateStorage.mkdirs(path, acls);
+ }
+
+ @Override
+ public void delete_node(String path) {
+ stateStorage.delete_node(path);
+ }
+
+ @Override
+ public void set_ephemeral_node(String path, byte[] data, List<ACL> acls) {
+ stateStorage.set_ephemeral_node(path, data, acls);
+ }
+
+ @Override
+ public Integer get_version(String path, boolean watch) throws Exception {
+ return stateStorage.get_version(path, watch);
+ }
+
+ @Override
+ public boolean node_exists(String path, boolean watch) {
+ return stateStorage.node_exists(path, watch);
+ }
+
+ @Override
+ public List<String> get_children(String path, boolean watch) {
+ return stateStorage.get_children(path, watch);
+ }
+
+ @Override
+ public void close() {
+ stateStorage.close();
+ pacemakerClient.close();
+ }
+
+ @Override
+ public void set_data(String path, byte[] data, List<ACL> acls) {
+ stateStorage.set_data(path, data, acls);
+ }
+
+ @Override
+ public byte[] get_data(String path, boolean watch) {
+ return stateStorage.get_data(path, watch);
+ }
+
+ @Override
+ public APersistentMap get_data_with_version(String path, boolean watch) {
+ return stateStorage.get_data_with_version(path, watch);
+ }
+
+ @Override
+ public void set_worker_hb(String path, byte[] data, List<ACL> acls) {
+ int retry = maxRetries;
+ while (true) {
+ try {
+ HBPulse hbPulse = new HBPulse();
+ hbPulse.set_id(path);
+ hbPulse.set_details(data);
+ HBMessage message = new HBMessage(HBServerMessageType.SEND_PULSE, HBMessageData.pulse(hbPulse));
+ HBMessage response = pacemakerClient.send(message);
+ if (response.get_type() != HBServerMessageType.SEND_PULSE_RESPONSE) {
+ throw new HBExecutionException("Invalid Response Type");
+ }
+ LOG.debug("Successful set_worker_hb");
+ break;
+ } catch (Exception e) {
+ if (retry <= 0) {
+ throw Utils.wrapInRuntime(e);
+ }
+ LOG.error("{} Failed to set_worker_hb. Will make {} more attempts.", e.getMessage(), retry--);
+ }
+ }
+ }
+
+ @Override
+ public byte[] get_worker_hb(String path, boolean watch) {
+ int retry = maxRetries;
+ while (true) {
+ try {
+ HBMessage message = new HBMessage(HBServerMessageType.GET_PULSE, HBMessageData.path(path));
+ HBMessage response = pacemakerClient.send(message);
+ if (response.get_type() != HBServerMessageType.GET_PULSE_RESPONSE) {
+ throw new HBExecutionException("Invalid Response Type");
+ }
+ LOG.debug("Successful get_worker_hb");
+ return response.get_data().get_pulse().get_details();
+ } catch (Exception e) {
+ if (retry <= 0) {
+ throw Utils.wrapInRuntime(e);
+ }
+ LOG.error("{} Failed to get_worker_hb. Will make {} more attempts.", e.getMessage(), retry--);
+ }
+ }
+ }
+
+ @Override
+ public List<String> get_worker_hb_children(String path, boolean watch) {
+ int retry = maxRetries;
+ while (true) {
+ try {
+ HBMessage message = new HBMessage(HBServerMessageType.GET_PULSE, HBMessageData.path(path));
+ HBMessage response = pacemakerClient.send(message);
+ if (response.get_type() != HBServerMessageType.GET_ALL_NODES_FOR_PATH_RESPONSE) {
+ throw new HBExecutionException("Invalid Response Type");
+ }
+ LOG.debug("Successful get_worker_hb");
+ return response.get_data().get_nodes().get_pulseIds();
+ } catch (Exception e) {
+ if (retry <= 0) {
+ throw Utils.wrapInRuntime(e);
+ }
+ LOG.error("{} Failed to get_worker_hb_children. Will make {} more attempts.", e.getMessage(), retry--);
+ }
+ }
+ }
+
+ @Override
+ public void delete_worker_hb(String path) {
+ int retry = maxRetries;
+ while (true) {
+ try {
+ HBMessage message = new HBMessage(HBServerMessageType.GET_PULSE, HBMessageData.path(path));
+ HBMessage response = pacemakerClient.send(message);
+ if (response.get_type() != HBServerMessageType.DELETE_PATH_RESPONSE) {
+ throw new HBExecutionException("Invalid Response Type");
+ }
+ LOG.debug("Successful get_worker_hb");
+ break;
+ } catch (Exception e) {
+ if (retry <= 0) {
+ throw Utils.wrapInRuntime(e);
+ }
+ LOG.error("{} Failed to delete_worker_hb. Will make {} more attempts.", e.getMessage(), retry--);
+ }
+ }
+ }
+
+ @Override
+ public void add_listener(ConnectionStateListener listener) {
+ stateStorage.add_listener(listener);
+ }
+
+ @Override
+ public void sync_path(String path) {
+ stateStorage.sync_path(path);
+ }
+
+ @Override
+ public void delete_node_blobstore(String path, String nimbusHostPortInfo) {
+ stateStorage.delete_node_blobstore(path, nimbusHostPortInfo);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java b/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java
new file mode 100644
index 0000000..eafd2e7
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java
@@ -0,0 +1,64 @@
+/**
+ * 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 org.apache.storm.pacemaker.PacemakerClient;
+import org.apache.storm.utils.Utils;
+import org.apache.zookeeper.data.ACL;
+
+import java.util.List;
+
+public class PaceMakerStateStorageFactory implements StateStorageFactory {
+
+ private static final PaceMakerStateStorageFactory INSTANCE = new PaceMakerStateStorageFactory();
+ private static PaceMakerStateStorageFactory _instance = INSTANCE;
+
+ public static void setInstance(PaceMakerStateStorageFactory u) {
+ _instance = u;
+ }
+
+ public static void resetInstance() {
+ _instance = INSTANCE;
+ }
+
+ @Override
+ public IStateStorage mkStore(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) {
+ try {
+ return new PaceMakerStateStorage(initMakeClient(config), initZKstate(config, auth_conf, acls, context));
+ } catch (Exception e) {
+ throw Utils.wrapInRuntime(e);
+ }
+ }
+
+ public static IStateStorage initZKstate(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
+ return _instance.initZKstateImpl(config, auth_conf, acls, context);
+ }
+
+ public static PacemakerClient initMakeClient(APersistentMap config) {
+ return _instance.initMakeClientImpl(config);
+ }
+
+ public IStateStorage initZKstateImpl(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
+ return ClusterUtils.mkStateStorage(config, auth_conf, acls, context);
+ }
+
+ public PacemakerClient initMakeClientImpl(APersistentMap config) {
+ return new PacemakerClient(config);
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/jvm/org/apache/storm/cluster/StateStorage.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StateStorage.java b/storm-core/src/jvm/org/apache/storm/cluster/StateStorage.java
deleted file mode 100644
index 8895cd1..0000000
--- a/storm-core/src/jvm/org/apache/storm/cluster/StateStorage.java
+++ /dev/null
@@ -1,220 +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.cluster;
-
-import clojure.lang.APersistentMap;
-import clojure.lang.IFn;
-import java.util.List;
-
-import org.apache.curator.framework.state.ConnectionStateListener;
-import org.apache.storm.callback.ZKStateChangedCallback;
-import org.apache.zookeeper.data.ACL;
-
-/**
- * StateStorage provides the API for the pluggable state store used by the
- * Storm daemons. Data is stored in path/value format, and the store supports
- * listing sub-paths at a given path.
- * All data should be available across all nodes with eventual consistency.
- *
- * IMPORTANT NOTE:
- * Heartbeats have different api calls used to interact with them. The root
- * path (/) may or may not be the same as the root path for the other api calls.
- *
- * For example, performing these two calls:
- * set_data("/path", data, acls);
- * void set_worker_hb("/path", heartbeat, acls);
- * may or may not cause a collision in "/path".
- * Never use the same paths with the *_hb* methods as you do with the others.
- */
-public interface StateStorage {
-
- /**
- * Registers a callback function that gets called when CuratorEvents happen.
- * @param callback is a clojure IFn that accepts the type - translated to
- * clojure keyword as in zookeeper - and the path: (callback type path)
- * @return is an id that can be passed to unregister(...) to unregister the
- * callback.
- */
- String register(ZKStateChangedCallback callback);
-
- /**
- * Unregisters a callback function that was registered with register(...).
- * @param id is the String id that was returned from register(...).
- */
- void unregister(String id);
-
- /**
- * Path will be appended with a monotonically increasing integer, a new node
- * will be created there, and data will be put at that node.
- * @param path The path that the monotonically increasing integer suffix will
- * be added to.
- * @param data The data that will be written at the suffixed path's node.
- * @param acls The acls to apply to the path. May be null.
- * @return The path with the integer suffix appended.
- */
- String create_sequential(String path, byte[] data, List<ACL> acls);
-
- /**
- * Creates nodes for path and all its parents. Path elements are separated by
- * a "/", as in *nix filesystem notation. Equivalent to mkdir -p in *nix.
- * @param path The path to create, along with all its parents.
- * @param acls The acls to apply to the path. May be null.
- * @return path
- */
- void mkdirs(String path, List<ACL> acls);
-
- /**
- * Deletes the node at a given path, and any child nodes that may exist.
- * @param path The path to delete
- */
- void delete_node(String path);
-
- /**
- * Creates an ephemeral node at path. Ephemeral nodes are destroyed
- * by the store when the client disconnects.
- * @param path The path where a node will be created.
- * @param data The data to be written at the node.
- * @param acls The acls to apply to the path. May be null.
- */
- void set_ephemeral_node(String path, byte[] data, List<ACL> acls);
-
- /**
- * Gets the 'version' of the node at a path. Optionally sets a watch
- * on that node. The version should increase whenever a write happens.
- * @param path The path to get the version of.
- * @param watch Whether or not to set a watch on the path. Watched paths
- * emit events which are consumed by functions registered with the
- * register method. Very useful for catching updates to nodes.
- * @return The integer version of this node.
- */
- Integer get_version(String path, boolean watch) throws Exception;
-
- /**
- * Check if a node exists and optionally set a watch on the path.
- * @param path The path to check for the existence of a node.
- * @param watch Whether or not to set a watch on the path. Watched paths
- * emit events which are consumed by functions registered with the
- * register method. Very useful for catching updates to nodes.
- * @return Whether or not a node exists at path.
- */
- boolean node_exists(String path, boolean watch);
-
- /**
- * Get a list of paths of all the child nodes which exist immediately
- * under path.
- * @param path The path to look under
- * @param watch Whether or not to set a watch on the path. Watched paths
- * emit events which are consumed by functions registered with the
- * register method. Very useful for catching updates to nodes.
- * @return list of string paths under path.
- */
- List<String> get_children(String path, boolean watch);
-
- /**
- * Close the connection to the data store.
- */
- void close();
-
- /**
- * Set the value of the node at path to data.
- * @param path The path whose node we want to set.
- * @param data The data to put in the node.
- * @param acls The acls to apply to the path. May be null.
- */
- void set_data(String path, byte[] data, List<ACL> acls);
-
- /**
- * Get the data from the node at path
- * @param path The path to look under
- * @param watch Whether or not to set a watch on the path. Watched paths
- * emit events which are consumed by functions registered with the
- * register method. Very useful for catching updates to nodes.
- * @return The data at the node.
- */
- byte[] get_data(String path, boolean watch);
-
- /**
- * Get the data at the node along with its version. Data is returned
- * in an APersistentMap with clojure keyword keys :data and :version.
- * @param path The path to look under
- * @param watch Whether or not to set a watch on the path. Watched paths
- * emit events which are consumed by functions registered with the
- * register method. Very useful for catching updates to nodes.
- * @return An APersistentMap in the form {:data data :version version}
- */
- APersistentMap get_data_with_version(String path, boolean watch);
-
- /**
- * Write a worker heartbeat at the path.
- * @param path The path whose node we want to set.
- * @param data The data to put in the node.
- * @param acls The acls to apply to the path. May be null.
- */
- void set_worker_hb(String path, byte[] data, List<ACL> acls);
-
- /**
- * Get the heartbeat from the node at path
- * @param path The path to look under
- * @param watch Whether or not to set a watch on the path. Watched paths
- * emit events which are consumed by functions registered with the
- * register method. Very useful for catching updates to nodes.
- * @return The heartbeat at the node.
- */
- byte[] get_worker_hb(String path, boolean watch);
-
- /**
- * Get a list of paths of all the child nodes which exist immediately
- * under path. This is similar to get_children, but must be used for
- * any nodes
- * @param path The path to look under
- * @param watch Whether or not to set a watch on the path. Watched paths
- * emit events which are consumed by functions registered with the
- * register method. Very useful for catching updates to nodes.
- * @return list of string paths under path.
- */
- List<String> get_worker_hb_children(String path, boolean watch);
-
- /**
- * Deletes the heartbeat at a given path, and any child nodes that may exist.
- * @param path The path to delete.
- */
- void delete_worker_hb(String path);
-
- /**
- * Add a StateStorageListener to the connection.
- * @param listener A StateStorageListener to handle changing cluster state
- * events.
- */
- void add_listener(final ConnectionStateListener listener);
-
- /**
- * Force consistency on a path. Any writes committed on the path before
- * this call will be completely propagated when it returns.
- * @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/2ee8bec8/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java b/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
index 9803dff..c2477d6 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
@@ -23,6 +23,6 @@ import org.apache.zookeeper.data.ACL;
public interface StateStorageFactory {
- StateStorage mkState(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context);
+ IStateStorage mkStore(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context);
}
http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
deleted file mode 100644
index 58b125b..0000000
--- a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
+++ /dev/null
@@ -1,129 +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.cluster;
-
-import clojure.lang.APersistentMap;
-import clojure.lang.IFn;
-import org.apache.storm.generated.*;
-import org.apache.storm.nimbus.NimbusInfo;
-
-import java.security.NoSuchAlgorithmException;
-import java.util.List;
-import java.util.Map;
-
-public interface StormClusterState {
- public List<String> assignments(IFn callback);
-
- public Assignment assignmentInfo(String stormId, IFn callback);
-
- public APersistentMap assignmentInfoWithVersion(String stormId, IFn callback);
-
- public Integer assignmentVersion(String stormId, IFn callback) throws Exception;
-
- // returns key information under /storm/blobstore/key
- public List<String> blobstoreInfo(String blobKey);
-
- // returns list of nimbus summaries stored under /stormroot/nimbuses/<nimbus-ids> -> <data>
- public List nimbuses();
-
- // adds the NimbusSummary to /stormroot/nimbuses/nimbus-id
- public void addNimbusHost(String nimbusId, NimbusSummary nimbusSummary);
-
- public List<String> activeStorms();
-
- public StormBase stormBase(String stormId, IFn callback);
-
- public ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, String node, Long port);
-
- public List<ProfileRequest> getWorkerProfileRequests(String stormId, NodeInfo nodeInfo, boolean isThrift);
-
- public List<ProfileRequest> getTopologyProfileRequests(String stormId, boolean isThrift);
-
- public void setWorkerProfileRequest(String stormId, ProfileRequest profileRequest);
-
- public void deleteTopologyProfileRequests(String stormId, ProfileRequest profileRequest);
-
- public Map<ExecutorInfo, ClusterWorkerHeartbeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort);
-
- public List<String> supervisors(IFn callback);
-
- public SupervisorInfo supervisorInfo(String supervisorId); // returns nil if doesn't exist
-
- public void setupHeatbeats(String stormId);
-
- public void teardownHeartbeats(String stormId);
-
- public void teardownTopologyErrors(String stormId);
-
- public List<String> heartbeatStorms();
-
- public List<String> errorTopologies();
-
- public void setTopologyLogConfig(String stormId, LogConfig logConfig);
-
- public LogConfig topologyLogConfig(String stormId, IFn cb);
-
- public void workerHeartbeat(String stormId, String node, Long port, ClusterWorkerHeartbeat info);
-
- public void removeWorkerHeartbeat(String stormId, String node, Long port);
-
- public void supervisorHeartbeat(String supervisorId, SupervisorInfo info);
-
- public void workerBackpressure(String stormId, String node, Long port, boolean on);
-
- public boolean topologyBackpressure(String stormId, IFn callback);
-
- public void setupBackpressure(String stormId);
-
- public void removeWorkerBackpressure(String stormId, String node, Long port);
-
- public void activateStorm(String stormId, StormBase stormBase);
-
- public void updateStorm(String stormId, StormBase newElems);
-
- public void removeStormBase(String stormId);
-
- public void setAssignment(String stormId, Assignment info);
-
- // sets up information related to key consisting of nimbus
- // host:port and version info of the blob
- public void setupBlobstore(String key, NimbusInfo nimbusInfo, Integer versionInfo);
-
- public List<String> activeKeys();
-
- public List<String> blobstore(IFn callback);
-
- public void removeStorm(String stormId);
-
- public void removeBlobstoreKey(String blobKey);
-
- public void removeKeyVersion(String blobKey);
-
- public void reportError(String stormId, String componentId, String node, Long port, String error);
-
- public List<ErrorInfo> errors(String stormId, String componentId);
-
- public ErrorInfo lastError(String stormId, String componentId);
-
- public void setCredentials(String stormId, Credentials creds, Map topoConf) throws NoSuchAlgorithmException;
-
- public Credentials credentials(String stormId, IFn callback);
-
- public void disconnect();
-
-}
http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
index cd2bc4a..8df5885 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
@@ -39,11 +39,11 @@ import java.util.*;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicReference;
-public class StormClusterStateImpl implements StormClusterState {
+public class StormClusterStateImpl implements IStormClusterState {
private static Logger LOG = LoggerFactory.getLogger(StormClusterStateImpl.class);
- private StateStorage stateStorage;
+ private IStateStorage stateStorage;
private ConcurrentHashMap<String, IFn> assignmentInfoCallback;
private ConcurrentHashMap<String, IFn> assignmentInfoWithVersionCallback;
@@ -61,7 +61,7 @@ public class StormClusterStateImpl implements StormClusterState {
private String stateId;
private boolean solo;
- public StormClusterStateImpl(StateStorage StateStorage, List<ACL> acls, ClusterStateContext context, boolean solo) throws Exception {
+ public StormClusterStateImpl(IStateStorage StateStorage, List<ACL> acls, ClusterStateContext context, boolean solo) throws Exception {
this.stateStorage = StateStorage;
this.solo = solo;
@@ -615,7 +615,7 @@ public class StormClusterStateImpl implements StormClusterState {
}
Collections.sort(errorInfos, new Comparator<ErrorInfo>() {
public int compare(ErrorInfo arg0, ErrorInfo arg1) {
- return -Integer.compare(arg0.get_error_time_secs(), arg1.get_error_time_secs());
+ return Integer.compare(arg1.get_error_time_secs(), arg0.get_error_time_secs());
}
});
} catch (Exception e) {
http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
index 8ac0adc..b277751 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
@@ -41,7 +41,7 @@ import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicBoolean;
-public class ZKStateStorage implements StateStorage {
+public class ZKStateStorage implements IStateStorage {
private static Logger LOG = LoggerFactory.getLogger(ZKStateStorage.class);
@@ -126,7 +126,7 @@ public class ZKStateStorage implements StateStorage {
@Override
public void delete_node_blobstore(String path, String nimbusHostPortInfo) {
-
+ Zookeeper.deleteNodeBlobstore(zkWriter, path, nimbusHostPortInfo);
}
@Override
http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
index 19b04f2..f3b9253 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
@@ -26,7 +26,7 @@ import java.util.List;
public class ZKStateStorageFactory implements StateStorageFactory{
@Override
- public StateStorage mkState(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) {
+ public IStateStorage mkStore(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) {
try {
return new ZKStateStorage(config, auth_conf, acls, context);
}catch (Exception e){
http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java b/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
index 34f3665..af0e8f3 100644
--- a/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
+++ b/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
@@ -157,7 +157,6 @@ public class PacemakerClient implements ISaslClient {
public String secretKey() {
return secret;
}
-
public HBMessage send(HBMessage m) {
waitUntilReady();
LOG.debug("Sending message: {}", m.toString());
http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedPaceMakerStateStorageFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedPaceMakerStateStorageFactory.java b/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedPaceMakerStateStorageFactory.java
new file mode 100644
index 0000000..0253afb
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedPaceMakerStateStorageFactory.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 org.apache.storm.testing.staticmocking;
+
+
+import org.apache.storm.cluster.PaceMakerStateStorageFactory;
+
+public class MockedPaceMakerStateStorageFactory implements AutoCloseable{
+
+ public MockedPaceMakerStateStorageFactory(PaceMakerStateStorageFactory inst) {
+ PaceMakerStateStorageFactory.setInstance(inst);
+ }
+
+ @Override
+ public void close() throws Exception {
+ PaceMakerStateStorageFactory.resetInstance();
+ }
+}
http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/test/clj/org/apache/storm/cluster_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/cluster_test.clj b/storm-core/test/clj/org/apache/storm/cluster_test.clj
index fa34355..39adb9e 100644
--- a/storm-core/test/clj/org/apache/storm/cluster_test.clj
+++ b/storm-core/test/clj/org/apache/storm/cluster_test.clj
@@ -23,7 +23,7 @@
(:import [org.mockito.exceptions.base MockitoAssertionError])
(:import [org.apache.curator.framework CuratorFramework CuratorFrameworkFactory CuratorFrameworkFactory$Builder])
(:import [org.apache.storm.utils Utils TestUtils ZookeeperAuthInfo ConfigUtils])
- (:import [org.apache.storm.cluster StateStorage ZKStateStorage ClusterStateContext StormClusterStateImpl ClusterUtils])
+ (:import [org.apache.storm.cluster IStateStorage ZKStateStorage ClusterStateContext StormClusterStateImpl ClusterUtils])
(:import [org.apache.storm.zookeeper Zookeeper])
(:import [org.apache.storm.callback ZKStateChangedCallback])
(:import [org.apache.storm.testing.staticmocking MockedZookeeper MockedCluster])
@@ -39,7 +39,7 @@
(defn mk-state
([zk-port] (let [conf (mk-config zk-port)]
- (ClusterUtils/mkDistributedClusterState conf conf nil (ClusterStateContext.))))
+ (ClusterUtils/mkStateStorage conf conf nil (ClusterStateContext.))))
([zk-port cb]
(let [ret (mk-state zk-port)]
(.register ret cb)
@@ -318,12 +318,12 @@
;; No need for when clauses because we just want to return nil
(with-open [_ (MockedZookeeper. zk-mock)]
(. (Mockito/when (.mkClientImpl zk-mock (Mockito/anyMap) (Mockito/anyList) (Mockito/any) (Mockito/anyString) (Mockito/any) (Mockito/anyMap))) (thenReturn curator-frameworke))
- (ClusterUtils/mkDistributedClusterState {} nil nil (ClusterStateContext.))
+ (ClusterUtils/mkStateStorage {} nil nil (ClusterStateContext.))
(.mkdirsImpl (Mockito/verify zk-mock (Mockito/times 1)) (Mockito/any) (Mockito/anyString) (Mockito/eq nil))))
- (let [distributed-state-storage (reify StateStorage
+ (let [distributed-state-storage (reify IStateStorage
(register [this callback] nil)
(mkdirs [this path acls] nil))
cluster-utils (Mockito/mock ClusterUtils)]
(with-open [mocked-cluster (MockedCluster. cluster-utils)]
- (. (Mockito/when (.mkDistributedClusterStateImpl cluster-utils (Mockito/any) (Mockito/any) (Mockito/eq nil) (Mockito/any))) (thenReturn distributed-state-storage))
+ (. (Mockito/when (mkStateStorageImpl cluster-utils (Mockito/any) (Mockito/any) (Mockito/eq nil) (Mockito/any))) (thenReturn distributed-state-storage))
(ClusterUtils/mkStormClusterState {} nil (ClusterStateContext.))))))
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj b/storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
index 0925237..1a7bd2c 100644
--- a/storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
+++ b/storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
@@ -15,13 +15,14 @@
;; limitations under the License.
(ns org.apache.storm.pacemaker-state-factory-test
(:require [clojure.test :refer :all]
- [conjure.core :refer :all]
- [org.apache.storm.pacemaker [pacemaker-state-factory :as psf]])
+ [conjure.core :refer :all])
(:import [org.apache.storm.generated
HBExecutionException HBNodes HBRecords
HBServerMessageType HBMessage HBMessageData HBPulse]
- [org.apache.storm.cluster ClusterStateContext]
- [org.mockito Mockito Matchers]))
+ [org.apache.storm.cluster ClusterStateContext PaceMakerStateStorageFactory]
+ [org.mockito Mockito Matchers])
+(:import [org.mockito.exceptions.base MockitoAssertionError])
+(:import [org.apache.storm.testing.staticmocking MockedPaceMakerStateStorageFactory]))
(defn- string-to-bytes [string]
(byte-array (map int string)))
@@ -39,18 +40,20 @@
(send [this something] (reset! captured something) response)
(check-captured [this] @captured))))
-(defmacro with-mock-pacemaker-client-and-state [client state response & body]
- `(let [~client (make-send-capture ~response)]
- (stubbing [psf/makeZKState nil
- psf/makeClient ~client]
- (let [~state (psf/-mkState nil nil nil nil (ClusterStateContext.))]
- ~@body))))
+(defmacro with-mock-pacemaker-client-and-state [client state pacefactory mock response & body]
+ `(let [~client (make-send-capture ~response)
+ ~pacefactory (Mockito/mock PaceMakerStateStorageFactory)]
+ (with-open [~mock (MockedPaceMakerStateStorageFactory. ~pacefactory)]
+ (. (Mockito/when (.initZKstateImpl ~pacefactory (Mockito/any) (Mockito/any) (Mockito/anyList) (Mockito/any))) (thenReturn nil))
+ (. (Mockito/when (.initMakeClientImpl ~pacefactory (Mockito/any))) (thenReturn ~client))
+ (let [~state (.mkStore ~pacefactory nil nil nil (ClusterStateContext.))]
+ ~@body))))
(deftest pacemaker_state_set_worker_hb
(testing "set_worker_hb"
(with-mock-pacemaker-client-and-state
- client state
+ client state pacefactory mock
(HBMessage. HBServerMessageType/SEND_PULSE_RESPONSE nil)
(.set_worker_hb state "/foo" (string-to-bytes "data") nil)
@@ -62,10 +65,10 @@
(testing "set_worker_hb"
(with-mock-pacemaker-client-and-state
- client state
+ client state pacefactory mock
(HBMessage. HBServerMessageType/SEND_PULSE nil)
- (is (thrown? HBExecutionException
+ (is (thrown? RuntimeException
(.set_worker_hb state "/foo" (string-to-bytes "data") nil))))))
@@ -73,7 +76,7 @@
(deftest pacemaker_state_delete_worker_hb
(testing "delete_worker_hb"
(with-mock-pacemaker-client-and-state
- client state
+ client state pacefactory mock
(HBMessage. HBServerMessageType/DELETE_PATH_RESPONSE nil)
(.delete_worker_hb state "/foo/bar")
@@ -83,16 +86,16 @@
(testing "delete_worker_hb"
(with-mock-pacemaker-client-and-state
- client state
+ client state pacefactory mock
(HBMessage. HBServerMessageType/DELETE_PATH nil)
- (is (thrown? HBExecutionException
+ (is (thrown? RuntimeException
(.delete_worker_hb state "/foo/bar"))))))
(deftest pacemaker_state_get_worker_hb
(testing "get_worker_hb"
(with-mock-pacemaker-client-and-state
- client state
+ client state pacefactory mock
(HBMessage. HBServerMessageType/GET_PULSE_RESPONSE
(HBMessageData/pulse
(doto (HBPulse.)
@@ -106,24 +109,24 @@
(testing "get_worker_hb - fail (bad response)"
(with-mock-pacemaker-client-and-state
- client state
+ client state pacefactory mock
(HBMessage. HBServerMessageType/GET_PULSE nil)
- (is (thrown? HBExecutionException
+ (is (thrown? RuntimeException
(.get_worker_hb state "/foo" false)))))
(testing "get_worker_hb - fail (bad data)"
(with-mock-pacemaker-client-and-state
- client state
+ client state pacefactory mock
(HBMessage. HBServerMessageType/GET_PULSE_RESPONSE nil)
- (is (thrown? HBExecutionException
+ (is (thrown? RuntimeException
(.get_worker_hb state "/foo" false))))))
(deftest pacemaker_state_get_worker_hb_children
(testing "get_worker_hb_children"
(with-mock-pacemaker-client-and-state
- client state
+ client state pacefactory mock
(HBMessage. HBServerMessageType/GET_ALL_NODES_FOR_PATH_RESPONSE
(HBMessageData/nodes
(HBNodes. [])))
@@ -135,16 +138,16 @@
(testing "get_worker_hb_children - fail (bad response)"
(with-mock-pacemaker-client-and-state
- client state
+ client state pacefactory mock
(HBMessage. HBServerMessageType/DELETE_PATH nil)
- (is (thrown? HBExecutionException
+ (is (thrown? RuntimeException
(.get_worker_hb_children state "/foo" false)))))
(testing "get_worker_hb_children - fail (bad data)"
(with-mock-pacemaker-client-and-state
- client state
+ client state pacefactory mock
(HBMessage. HBServerMessageType/GET_ALL_NODES_FOR_PATH_RESPONSE nil)
-
- (is (thrown? HBExecutionException
+ ;need been update due to HBExecutionException
+ (is (thrown? RuntimeException
(.get_worker_hb_children state "/foo" false))))))
[07/27] storm git commit: update class hierarchy about cluster
Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/test/clj/org/apache/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/nimbus_test.clj b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
index d4402fb..772a232 100644
--- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj
+++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
@@ -36,7 +36,7 @@
(:import [org.apache.storm.utils Time Utils ConfigUtils])
(:import [org.apache.storm.zookeeper Zookeeper])
(:import [org.apache.commons.io FileUtils])
- (:import [org.apache.storm.cluster StormZkClusterState ClusterStateContext Cluster])
+ (:import [org.apache.storm.cluster StormClusterStateImpl ClusterStateContext ClusterUtils])
(:use [org.apache.storm testing MockAutoCred util config log timer converter])
(:use [org.apache.storm.daemon common])
(:require [conjure.core])
@@ -126,7 +126,7 @@
(let [state (:storm-cluster-state cluster)
executor->node+port (:executor->node+port (clojurify-assignment (.assignmentInfo state storm-id nil)))
[node port] (get executor->node+port executor)
- curr-beat (clojurify-zk-worker-hb (.getworkerHeartbeat state storm-id node port))
+ curr-beat (clojurify-zk-worker-hb (.getWorkerHeartbeat state storm-id node port))
stats (:executor-stats curr-beat)]
(.workerHeartbeat state storm-id node port
(thriftify-zk-worker-hb {:storm-id storm-id :time-secs (current-time-secs) :uptime 10 :executor-stats (merge stats {executor (stats/render-stats! (stats/mk-bolt-stats 20))})})
@@ -1031,7 +1031,7 @@
STORM-CLUSTER-MODE "local"
STORM-ZOOKEEPER-PORT zk-port
STORM-LOCAL-DIR nimbus-dir}))
- (bind cluster-state (StormZkClusterState. conf nil (ClusterStateContext.)))
+ (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
(bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
(bind topology (thrift/mk-topology
{"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
@@ -1103,7 +1103,7 @@
STORM-CLUSTER-MODE "local"
STORM-ZOOKEEPER-PORT zk-port
STORM-LOCAL-DIR nimbus-dir}))
- (bind cluster-state (StormZkClusterState. conf nil (ClusterStateContext.)))
+ (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
(bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
(bind topology (thrift/mk-topology
{"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
@@ -1113,7 +1113,7 @@
(zkLeaderElectorImpl [conf] (mock-leader-elector :is-leader false))))]
(letlocals
- (bind non-leader-cluster-state (StormZkClusterState. conf nil (ClusterStateContext.)))
+ (bind non-leader-cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
(bind non-leader-nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
;first we verify that the master nimbus can perform all actions, even with another nimbus present.
@@ -1309,7 +1309,7 @@
:status {:type bogus-type}}
}
]
- (stubbing [topology-bases bogus-bases
+ (stubbing [nimbus/nimbus-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.
@@ -1350,14 +1350,13 @@
NIMBUS-THRIFT-PORT 6666})
expected-acls nimbus/NIMBUS-ZK-ACLS
fake-inimbus (reify INimbus (getForcedScheduler [this] nil))
- storm-zk (Mockito/mock Cluster)]
+ cluster-utils (Mockito/mock ClusterUtils)]
(with-open [_ (proxy [MockedConfigUtils] []
(nimbusTopoHistoryStateImpl [conf] nil))
zk-le (MockedZookeeper. (proxy [Zookeeper] []
(zkLeaderElectorImpl [conf] nil)))
- storm-zk-le (MockedCluster. storm-zk)]
+ mocked-cluster (MockedCluster. cluster-utils)]
(stubbing [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
@@ -1366,10 +1365,7 @@
mk-timer nil
nimbus/mk-scheduler nil]
(nimbus/nimbus-data auth-conf fake-inimbus)
- (.mkStormClusterStateImpl (Mockito/verify storm-zk (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
- ; (verify-call-times-for cluster/mk-storm-cluster-state 1)
- ; (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
- ; expected-acls)
+ (.mkStormClusterStateImpl (Mockito/verify cluster-utils (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
)))))
(deftest test-file-bogus-download
@@ -1401,7 +1397,7 @@
STORM-CLUSTER-MODE "local"
STORM-ZOOKEEPER-PORT zk-port
STORM-LOCAL-DIR nimbus-dir}))
- (bind cluster-state (StormZkClusterState. conf nil (ClusterStateContext.)))
+ (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
(bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
(sleep-secs 1)
(bind topology (thrift/mk-topology
@@ -1433,7 +1429,7 @@
STORM-ZOOKEEPER-PORT zk-port
STORM-LOCAL-DIR nimbus-dir
NIMBUS-TOPOLOGY-ACTION-NOTIFIER-PLUGIN (.getName InMemoryTopologyActionNotifier)}))
- (bind cluster-state (StormZkClusterState. conf nil (ClusterStateContext.)))
+ (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
(bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
(bind notifier (InMemoryTopologyActionNotifier.))
(sleep-secs 1)
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/test/clj/org/apache/storm/supervisor_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/supervisor_test.clj b/storm-core/test/clj/org/apache/storm/supervisor_test.clj
index c98a68b..b89b7bb 100644
--- a/storm-core/test/clj/org/apache/storm/supervisor_test.clj
+++ b/storm-core/test/clj/org/apache/storm/supervisor_test.clj
@@ -29,7 +29,7 @@
(:import [org.mockito.exceptions.base MockitoAssertionError])
(:import [java.io File])
(:import [java.nio.file Files])
- (:import [org.apache.storm.cluster StormZkClusterState Cluster ClusterStateContext])
+ (:import [org.apache.storm.cluster StormClusterStateImpl ClusterStateContext ClusterUtils])
(:import [java.nio.file.attribute FileAttribute])
(:use [org.apache.storm config testing util timer log converter])
(:use [org.apache.storm.daemon common])
@@ -565,17 +565,17 @@
fake-isupervisor (reify ISupervisor
(getSupervisorId [this] nil)
(getAssignmentId [this] nil))
- storm-zk (Mockito/mock Cluster)]
+ cluster-utils (Mockito/mock ClusterUtils)]
(with-open [_ (proxy [MockedConfigUtils] []
(supervisorStateImpl [conf] nil)
(supervisorLocalDirImpl [conf] nil))
- storm-zk-le (MockedCluster. storm-zk)]
+ mocked-cluster (MockedCluster. cluster-utils)]
(stubbing [uptime-computer nil
; cluster/mk-storm-cluster-state nil
local-hostname nil
mk-timer nil]
(supervisor/supervisor-data auth-conf nil fake-isupervisor)
- (.mkStormClusterStateImpl (Mockito/verify storm-zk (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
+ (.mkStormClusterStateImpl (Mockito/verify cluster-utils (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
; (verify-call-times-for cluster/mk-storm-cluster-state 1)
; (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
; expected-acls)
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/test/jvm/org/apache/storm/ClusterTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/ClusterTest.java b/storm-core/test/jvm/org/apache/storm/ClusterTest.java
deleted file mode 100644
index ef43afe..0000000
--- a/storm-core/test/jvm/org/apache/storm/ClusterTest.java
+++ /dev/null
@@ -1,22 +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;
-
-
-public class ClusterTest {
-}
[27/27] storm git commit: Added STORM-1273 to Changelog
Posted by bo...@apache.org.
Added STORM-1273 to Changelog
Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/71d615b7
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/71d615b7
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/71d615b7
Branch: refs/heads/master
Commit: 71d615b7cc9a96b6667b976a25dc86ef54a66169
Parents: 1b5d9a8
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Wed Feb 24 09:46:03 2016 -0600
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Wed Feb 24 09:46:03 2016 -0600
----------------------------------------------------------------------
CHANGELOG.md | 1 +
1 file changed, 1 insertion(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/storm/blob/71d615b7/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index feef9a3..cc11139 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
## 2.0.0
+ * STORM-1273: port backtype.storm.cluster to java
* STORM-1479: use a simple implemention for IntSerializer
* STORM-1255: port storm_utils.clj to java and split Time tests into its
* STORM-1566: Worker exits with error o.a.s.d.worker [ERROR] Error on initialization of server mk-worker