From 682d31c8a5f53a59f47b4df4bd35ff828e7e0aa5 Mon Sep 17 00:00:00 2001 From: "xiaojian.fxj" Date: Mon, 1 Feb 2016 15:17:49 +0800 Subject: [PATCH 01/10] port org.apache.storm.cluster.cluster.clj --- .../jvm/org/apache/storm/cluster/Cluster.java | 204 ++++++ .../apache/storm/cluster/ClusterState.java | 11 +- .../cluster/DistributedClusterState.java | 269 +++++++ .../storm/cluster/StormClusterState.java | 129 ++++ .../storm/cluster/StormZkClusterState.java | 662 ++++++++++++++++++ 5 files changed, 1271 insertions(+), 4 deletions(-) create mode 100644 storm-core/src/jvm/org/apache/storm/cluster/Cluster.java create mode 100644 storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java create mode 100644 storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java create mode 100644 storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.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 00000000000..2d6f3069b20 --- /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 mkTopoOnlyAcls(Map topoConf) throws NoSuchAlgorithmException { + List 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 maybeDeserialize(byte[] serialized, Class 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 convertExecutorBeats(List executors, ClusterWorkerHeartbeat workerHeartbeat){ + Map executorWhb = new HashMap<>(); + Map executorStatsMap = workerHeartbeat.get_executor_stats(); + for (ExecutorInfo executor : executors){ + if(executorStatsMap.containsKey(executor)){ + executorWhb.put(executor, workerHeartbeat); + } + } + return executorWhb; + } + + // TO be remove + public static HashMap> reverseMap(Map map) { + HashMap> rtn = new HashMap>(); + if (map == null) { + return rtn; + } + for (Map.Entry entry : map.entrySet()) { + K key = entry.getKey(); + V val = entry.getValue(); + List list = rtn.get(val); + if (list == null) { + list = new ArrayList(); + rtn.put(entry.getValue(), list); + } + list.add(key); + } + return rtn; + } +} 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 fdac92c7a3d..51e42fff969 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 @@ 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 acls); + void mkdirs(String path, List 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 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 00000000000..3e0beb11207 --- /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 callbacks = new ConcurrentHashMap(); + private CuratorFramework zkWriter; + private CuratorFramework zkReader; + private AtomicBoolean active; + + private boolean isNimbus; + private Map authConf; + private Map conf; + + public DistributedClusterState(Map conf, Map authConf, List 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 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 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) 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) 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 acls) { + return Zookeeper.createNode(zkWriter, path, data, CreateMode.EPHEMERAL_SEQUENTIAL, acls); + } + + @Override + public void mkdirs(String path, List 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 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 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 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 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 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 toks = Zookeeper.tokenizePath(path); + int size = toks.size(); + if (size > 0) { + toks.remove(size - 1); + } + return Zookeeper.toksToPath(toks); + } +} 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 00000000000..b3c0f90dfb9 --- /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 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 blobstoreInfo(String blobKey); + + // returns list of nimbus summaries stored under /stormroot/nimbuses/ -> + public List nimbuses(); + + // adds the NimbusSummary to /stormroot/nimbuses/nimbus-id + public void addNimbusHost(String nimbusId, NimbusSummary nimbusSummary); + + public List activeStorms(); + + public StormBase stormBase(String stormId, Callback callback); + + public ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, String node, Long port); + + public List getWorkerProfileRequets(String stormId, NodeInfo nodeInfo, boolean isThrift); + + public List getTopologyProfileRequets(String stormId, boolean isThrift); + + public void setWorkerProfileRequests(String stormId, ProfileRequest profileRequest); + + public void deleteTopologyProfileRequests(String stormId, ProfileRequest profileRequest); + + public Map executorBeats(String stormId, Map, NodeInfo> executorNodePort); + + public List 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 heartbeatStorms(); + + public List 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 activeKeys(); + + public List 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 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(); + +} 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 00000000000..93d29b2d57c --- /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 assignmentInfoCallback; + private ConcurrentHashMap assignmentInfoWithVersionCallback; + private ConcurrentHashMap assignmentVersionCallback; + private AtomicReference supervisorsCallback; + // we want to reigister a topo directory getChildren callback for all workers of this dir + private ConcurrentHashMap backPressureCallback; + private AtomicReference assignmentsCallback; + private ConcurrentHashMap stormBaseCallback; + private AtomicReference blobstoreCallback; + private ConcurrentHashMap credentialsCallback; + private ConcurrentHashMap logConfigCallback; + + private List acls; + private String stateId; + private boolean solo; + + public StormZkClusterState(Object clusterState, List 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 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 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 cb) { + Callback callback = cb.getAndSet(null); + callback.execute(); + } + + protected void issueMapCallback(ConcurrentHashMap callbackConcurrentHashMap, String key) { + Callback callback = callbackConcurrentHashMap.remove(key); + callback.execute(); + } + + @Override + public List 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 blobstoreInfo(String blobKey) { + String path = Cluster.blobstorePath(blobKey); + clusterState.sync_path(path); + return clusterState.get_children(path, false); + } + + @Override + public List nimbuses() { + List nimbusSummaries = new ArrayList<>(); + List 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 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 getWorkerProfileRequets(String stormId, NodeInfo nodeInfo, boolean isThrift) { + List requests = new ArrayList<>(); + List 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 getTopologyProfileRequets(String stormId, boolean isThrift) { + List profileRequests = new ArrayList<>(); + String path = Cluster.profilerConfigPath(stormId); + if (clusterState.node_exists(path, false)) { + List 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 executorBeats(String stormId, Map, NodeInfo> executorNodePort) { + Map executorWhbs = new HashMap<>(); + + Map>> nodePortExecutors = Cluster.reverseMap(executorNodePort); + for (Map.Entry>> entry : nodePortExecutors.entrySet()) { + + String node = entry.getKey().get_node(); + Long port = entry.getKey().get_port_iterator().next(); + ClusterWorkerHeartbeat whb = getWorkerHeartbeat(stormId, node, port); + List executorInfoList = new ArrayList<>(); + for (List 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 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 heartbeatStorms() { + return clusterState.get_worker_hb_children(Cluster.WORKERBEATS_SUBTREE, false); + } + + @Override + public List 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 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 componentExecutors = newElems.get_component_executors(); + if (componentExecutors == null) { + componentExecutors = new HashMap<>(); + } + for (Map.Entry 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 ComponentDebug = new HashMap<>(); + Map oldComponentDebug = stormBase.get_component_debug(); + if (oldComponentDebug == null) + oldComponentDebug = new HashMap<>(); + Map newComponentDebug = newElems.get_component_debug(); + if (newComponentDebug == null) + newComponentDebug = new HashMap<>(); + Set 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 activeKeys() { + return clusterState.get_children(Cluster.BLOBSTORE_SUBTREE, false); + } + + // blobstore state + @Override + public List 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 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 errors(String stormId, String componentId) { + List errorInfos = new ArrayList<>(); + try { + String path = Cluster.errorPath(stormId, componentId); + if (clusterState.node_exists(path, false)) { + List 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() { + 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 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(); + } +} From 9a79fb7de0e824e73c294738521e892f1d81fbb0 Mon Sep 17 00:00:00 2001 From: "xiaojian.fxj" Date: Wed, 3 Feb 2016 20:28:05 +0800 Subject: [PATCH 02/10] delete zookeeper.clj zookeeper_state_factory.clj cluster.clj, but some tests still can't pass --- conf/defaults.yaml | 2 +- .../src/clj/org/apache/storm/cluster.clj | 691 ------------------ .../cluster_state/zookeeper_state_factory.clj | 163 ----- .../apache/storm/command/dev_zookeeper.clj | 2 +- .../org/apache/storm/command/heartbeats.clj | 6 +- .../apache/storm/command/shell_submission.clj | 2 +- .../src/clj/org/apache/storm/converter.clj | 14 +- .../clj/org/apache/storm/daemon/common.clj | 13 +- .../clj/org/apache/storm/daemon/executor.clj | 12 +- .../clj/org/apache/storm/daemon/nimbus.clj | 138 ++-- .../org/apache/storm/daemon/supervisor.clj | 35 +- .../clj/org/apache/storm/daemon/worker.clj | 43 +- .../pacemaker/pacemaker_state_factory.clj | 12 +- storm-core/src/clj/org/apache/storm/stats.clj | 3 +- .../src/clj/org/apache/storm/testing.clj | 14 +- .../src/clj/org/apache/storm/thrift.clj | 2 +- .../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 -- .../org/apache/storm/callback/Callback.java | 3 + .../jvm/org/apache/storm/cluster/Cluster.java | 38 +- .../apache/storm/cluster/ClusterState.java | 2 +- .../cluster/DistributedClusterState.java | 7 +- .../storm/cluster/StormClusterState.java | 34 +- .../storm/cluster/StormZkClusterState.java | 109 +-- .../testing/staticmocking/MockedCluster.java | 31 + .../org/apache/storm/integration_test.clj | 15 +- .../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 +- .../jvm/org/apache/storm/ClusterTest.java | 22 + 32 files changed, 488 insertions(+), 1296 deletions(-) delete mode 100644 storm-core/src/clj/org/apache/storm/cluster.clj delete mode 100644 storm-core/src/clj/org/apache/storm/cluster_state/zookeeper_state_factory.clj delete mode 100644 storm-core/src/clj/org/apache/storm/zookeeper.clj create mode 100644 storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedCluster.java create mode 100644 storm-core/test/jvm/org/apache/storm/ClusterTest.java diff --git a/conf/defaults.yaml b/conf/defaults.yaml index 8873d123925..74605bbc960 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" 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 152423afc0c..00000000000 --- 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/ -> - (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 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 dcfa8d83257..00000000000 --- 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)))))) 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 ef9ecbbf375..7be526d236e 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)) 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 be8d030f796..954042f32b6 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" 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 8a5eb213d3d..3978d2f9545 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)) diff --git a/storm-core/src/clj/org/apache/storm/converter.clj b/storm-core/src/clj/org/apache/storm/converter.clj index bb2dc8777e2..d1693018a70 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)) 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 6c184fd2f25..c9534f41a5d 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)] )) )) 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 82d56a941de..e50e15069c8 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)) 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 de5a14ea501..9b00df37519 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)) 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 337a1b4613a..079b22188f9 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) 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 48934f6538e..85ed37dab2f 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) 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 cede59e0941..b367b4baf66 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)) diff --git a/storm-core/src/clj/org/apache/storm/stats.clj b/storm-core/src/clj/org/apache/storm/stats.clj index 68b16fd2f07..d6bcdc30c7e 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 diff --git a/storm-core/src/clj/org/apache/storm/testing.clj b/storm-core/src/clj/org/apache/storm/testing.clj index cc786590e87..eb34d365259 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)] diff --git a/storm-core/src/clj/org/apache/storm/thrift.clj b/storm-core/src/clj/org/apache/storm/thrift.clj index b5af521010a..4dc21f9fb27 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] 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 f26d998d1a1..14313be368b 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 diff --git a/storm-core/src/clj/org/apache/storm/util.clj b/storm-core/src/clj/org/apache/storm/util.clj index 23d39f672c0..165d8ee07ce 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] 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 413ffd6571d..00000000000 --- 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)) - 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 29b97619817..a37612d28ff 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 Object execute(T... args); } 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 2d6f3069b20..851858ff6c8 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.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 mkTopoOnlyAcls(Map topoConf) throws NoSuchAlgorithmException { - List aclList = new ArrayList<>(); + List 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 static Map convertExecutorBeats(Lis } return executorWhb; } + + public StormClusterState mkStormClusterStateImpl(Object clusterState, List acls, ClusterStateContext context) throws Exception{ + return new StormZkClusterState(clusterState, acls, context); + } + public static StormClusterState mkStormClusterState(Object clusterState, List acls, ClusterStateContext context) throws Exception{ + return _instance.mkStormClusterStateImpl(clusterState, acls, context); + } // TO be remove public static HashMap> reverseMap(Map map) { 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 51e42fff969..e76721bd6fc 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. */ 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 3e0beb11207..1bd534e9cda 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 conf, Map authConf, List 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 void delete_node_blobstore(String path, String nimbusHostPortInfo) { } @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; } 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 b3c0f90dfb9..ede2ba368e3 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.Map; public interface StormClusterState { - public List assignments(Callback callback); + public List 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 blobstoreInfo(String blobKey); @@ -46,27 +46,27 @@ public interface StormClusterState { public List 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 getWorkerProfileRequets(String stormId, NodeInfo nodeInfo, boolean isThrift); + public List getWorkerProfileRequests(String stormId, NodeInfo nodeInfo, boolean isThrift); - public List getTopologyProfileRequets(String stormId, boolean isThrift); + public List 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 executorBeats(String stormId, Map, NodeInfo> executorNodePort); - public List supervisors(Callback callback); + public List 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 activeKeys(); - public List blobstore(Callback callback); + public List 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 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(); 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 93d29b2d57c..3f32fe1698c 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 assignmentInfoCallback; - private ConcurrentHashMap assignmentInfoWithVersionCallback; - private ConcurrentHashMap assignmentVersionCallback; - private AtomicReference supervisorsCallback; + private ConcurrentHashMap assignmentInfoCallback; + private ConcurrentHashMap assignmentInfoWithVersionCallback; + private ConcurrentHashMap assignmentVersionCallback; + private AtomicReference supervisorsCallback; // we want to reigister a topo directory getChildren callback for all workers of this dir - private ConcurrentHashMap backPressureCallback; - private AtomicReference assignmentsCallback; - private ConcurrentHashMap stormBaseCallback; - private AtomicReference blobstoreCallback; - private ConcurrentHashMap credentialsCallback; - private ConcurrentHashMap logConfigCallback; + private ConcurrentHashMap backPressureCallback; + private AtomicReference assignmentsCallback; + private ConcurrentHashMap stormBaseCallback; + private AtomicReference blobstoreCallback; + private ConcurrentHashMap credentialsCallback; + private ConcurrentHashMap logConfigCallback; private List acls; private String stateId; @@ -102,7 +103,7 @@ public Object execute(T... args) { 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 Object execute(T... args) { } - protected void issueCallback(AtomicReference cb) { - Callback callback = cb.getAndSet(null); - callback.execute(); + protected void issueCallback(AtomicReference cb) { + IFn callback = cb.getAndSet(null); + callback.invoke(); } - protected void issueMapCallback(ConcurrentHashMap callbackConcurrentHashMap, String key) { - Callback callback = callbackConcurrentHashMap.remove(key); - callback.execute(); + protected void issueMapCallback(ConcurrentHashMap callbackConcurrentHashMap, String key) { + IFn callback = callbackConcurrentHashMap.remove(key); + callback.invoke(); } @Override - public List assignments(Callback callback) { + public List assignments(IFn callback) { if (callback != null) { assignmentsCallback.set(callback); } @@ -164,7 +165,7 @@ public List assignments(Callback callback) { } @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 Assignment assignmentInfo(String stormId, Callback callback) { } @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 APersistentMap assignmentInfoWithVersion(String stormId, Callback callbac } @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 List activeStorms() { } @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 ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, String node, Lo } @Override - public List getWorkerProfileRequets(String stormId, NodeInfo nodeInfo, boolean isThrift) { + public List getWorkerProfileRequests(String stormId, NodeInfo nodeInfo, boolean isThrift) { List requests = new ArrayList<>(); - List profileRequests = getTopologyProfileRequets(stormId, isThrift); + List profileRequests = getTopologyProfileRequests(stormId, isThrift); for (ProfileRequest profileRequest : profileRequests) { NodeInfo nodeInfo1 = profileRequest.get_nodeInfo(); if (nodeInfo1.equals(nodeInfo)) @@ -266,7 +267,7 @@ public List getWorkerProfileRequets(String stormId, NodeInfo nod } @Override - public List getTopologyProfileRequets(String stormId, boolean isThrift) { + public List getTopologyProfileRequests(String stormId, boolean isThrift) { List profileRequests = new ArrayList<>(); String path = Cluster.profilerConfigPath(stormId); if (clusterState.node_exists(path, false)) { @@ -283,7 +284,7 @@ public List getTopologyProfileRequets(String stormId, boolean is } @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 void deleteTopologyProfileRequests(String stormId, ProfileRequest profile 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 executorBeats(String stormId, Map, NodeInfo> executorNodePort) { Map executorWhbs = new HashMap<>(); + LOG.info(executorNodePort.toString()); Map>> nodePortExecutors = Cluster.reverseMap(executorNodePort); + LOG.info(nodePortExecutors.toString()); + for (Map.Entry>> entry : nodePortExecutors.entrySet()) { String node = entry.getKey().get_node(); @@ -320,7 +328,7 @@ public Map executorBeats(String stormId, M } @Override - public List supervisors(Callback callback) { + public List supervisors(IFn callback) { if (callback != null) { supervisorsCallback.set(callback); } @@ -339,7 +347,7 @@ public void setupHeatbeats(String stormId) { } @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 void setTopologyLogConfig(String stormId, LogConfig logConfig) { } @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 void workerBackpressure(String stormId, String node, Long port, boolean o // 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 void updateStorm(String stormId, StormBase newElems) { StormBase stormBase = stormBase(stormId, null); if (stormBase.get_component_executors() != null) { + + Map newComponentExecutors = new HashMap<>(); Map componentExecutors = newElems.get_component_executors(); - if (componentExecutors == null) { - componentExecutors = new HashMap<>(); + //componentExecutors maybe be APersistentMap, which don't support put + for (Map.Entry entry : componentExecutors.entrySet()) { + newComponentExecutors.put(entry.getKey(), entry.getValue()); } for (Map.Entry 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 ComponentDebug = new HashMap<>(); Map oldComponentDebug = stormBase.get_component_debug(); - if (oldComponentDebug == null) - oldComponentDebug = new HashMap<>(); + Map newComponentDebug = newElems.get_component_debug(); - if (newComponentDebug == null) - newComponentDebug = new HashMap<>(); + Set debugOptionsKeys = oldComponentDebug.keySet(); debugOptionsKeys.addAll(newComponentDebug.keySet()); for (String key : debugOptionsKeys) { @@ -499,7 +508,17 @@ public void updateStorm(String stormId, StormBase newElems) { 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 void setAssignment(String stormId, Assignment info) { } @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 List activeKeys() { // blobstore state @Override - public List blobstore(Callback callback) { + public List blobstore(IFn callback) { if (callback != null) { blobstoreCallback.set(callback); } @@ -571,7 +590,7 @@ public void removeKeyVersion(String blobKey) { } @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 void setCredentials(String stormId, Credentials creds, Map topoConf) thro } @Override - public Credentials credentials(String stormId, Callback callback) { + public Credentials credentials(String stormId, IFn callback) { if (callback != null) { credentialsCallback.put(stormId, callback); } 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 00000000000..5d67a545142 --- /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(); + } +} 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 cd2bc266866..d374511019b 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 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 ffd913e7c07..d0b988217ca 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)))))) + 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 19c6f596442..d4402fb04bd 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) 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 361c4be7a3f..307296aa3eb 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]) 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 edb161bda37..c98a68bcf78 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" 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 00000000000..ef43afeaece --- /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 { +} From 5916b0b8089f9dd184fdd1ab2f18eb5e5deabc65 Mon Sep 17 00:00:00 2001 From: "xiaojian.fxj" Date: Wed, 3 Feb 2016 21:15:41 +0800 Subject: [PATCH 03/10] callback maybe null --- .../jvm/org/apache/storm/cluster/StormZkClusterState.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) 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 3f32fe1698c..3a4205b2664 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 Object execute(T... args) { protected void issueCallback(AtomicReference cb) { IFn callback = cb.getAndSet(null); - callback.invoke(); + if (callback != null) + callback.invoke(); } protected void issueMapCallback(ConcurrentHashMap callbackConcurrentHashMap, String key) { IFn callback = callbackConcurrentHashMap.remove(key); - callback.invoke(); + if (callback != null) + callback.invoke(); } @Override From 55b86ca4f0ea02b25701f25f454e537cbf6239d4 Mon Sep 17 00:00:00 2001 From: "xiaojian.fxj" Date: Fri, 5 Feb 2016 13:47:12 +0800 Subject: [PATCH 04/10] update class hierarchy about cluster --- conf/defaults.yaml | 2 +- .../org/apache/storm/command/heartbeats.clj | 4 +- .../clj/org/apache/storm/daemon/common.clj | 5 +- .../clj/org/apache/storm/daemon/executor.clj | 6 +- .../clj/org/apache/storm/daemon/nimbus.clj | 30 +- .../org/apache/storm/daemon/supervisor.clj | 12 +- .../clj/org/apache/storm/daemon/worker.clj | 20 +- .../pacemaker/pacemaker_state_factory.clj | 11 +- storm-core/src/clj/org/apache/storm/stats.clj | 2 +- .../src/clj/org/apache/storm/testing.clj | 6 +- ...lback.java => ZKStateChangedCallback.java} | 9 +- .../storm/cluster/ClusterStateContext.java | 2 +- .../{Cluster.java => ClusterUtils.java} | 121 ++++--- .../{ClusterState.java => StateStorage.java} | 12 +- ...eFactory.java => StateStorageFactory.java} | 4 +- .../storm/cluster/StormClusterState.java | 2 +- ...rState.java => StormClusterStateImpl.java} | 319 ++++++++---------- ...dClusterState.java => ZKStateStorage.java} | 41 +-- .../storm/cluster/ZKStateStorageFactory.java} | 18 +- .../testing/staticmocking/MockedCluster.java | 8 +- .../org/apache/storm/zookeeper/Zookeeper.java | 22 +- .../org/apache/storm/integration_test.clj | 4 +- .../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 +- 25 files changed, 422 insertions(+), 396 deletions(-) rename storm-core/src/jvm/org/apache/storm/callback/{Callback.java => ZKStateChangedCallback.java} (84%) rename storm-core/src/jvm/org/apache/storm/cluster/{Cluster.java => ClusterUtils.java} (63%) rename storm-core/src/jvm/org/apache/storm/cluster/{ClusterState.java => StateStorage.java} (96%) rename storm-core/src/jvm/org/apache/storm/cluster/{ClusterStateFactory.java => StateStorageFactory.java} (90%) rename storm-core/src/jvm/org/apache/storm/cluster/{StormZkClusterState.java => StormClusterStateImpl.java} (62%) rename storm-core/src/jvm/org/apache/storm/cluster/{DistributedClusterState.java => ZKStateStorage.java} (85%) rename storm-core/{test/jvm/org/apache/storm/ClusterTest.java => src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java} (59%) diff --git a/conf/defaults.yaml b/conf/defaults.yaml index 74605bbc960..b517b90f0d6 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" 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 954042f32b6..af86b699415 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" 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 c9534f41a5d..b144f402fbe 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)] )) 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 7c34c8f8bf8..49ae6cfa004 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) 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 9b00df37519..daf5e4558b9 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)) 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 079b22188f9..3a83d032af7 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) 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 85ed37dab2f..a79300957a2 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) 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 b367b4baf66..28f792d3c4f 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)) diff --git a/storm-core/src/clj/org/apache/storm/stats.clj b/storm-core/src/clj/org/apache/storm/stats.clj index d6bcdc30c7e..0bf1757791a 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]])) diff --git a/storm-core/src/clj/org/apache/storm/testing.clj b/storm-core/src/clj/org/apache/storm/testing.clj index eb34d365259..470a14f49b4 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 diff --git a/storm-core/src/jvm/org/apache/storm/callback/Callback.java b/storm-core/src/jvm/org/apache/storm/callback/ZKStateChangedCallback.java similarity index 84% rename from storm-core/src/jvm/org/apache/storm/callback/Callback.java rename to storm-core/src/jvm/org/apache/storm/callback/ZKStateChangedCallback.java index a37612d28ff..75b0e99453c 100644 --- a/storm-core/src/jvm/org/apache/storm/callback/Callback.java +++ b/storm-core/src/jvm/org/apache/storm/callback/ZKStateChangedCallback.java @@ -18,9 +18,8 @@ package org.apache.storm.callback; -import clojure.lang.IFn; +import org.apache.zookeeper.Watcher; -// To remove IFn after porting all callbacks to java -public interface Callback { - public Object execute(T... args); -} +public interface ZKStateChangedCallback { + public void changed(Watcher.Event.EventType type, String path); +} \ No newline at end of file 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 997bdc3406f..9ad6a92b87e 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 { diff --git a/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java b/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java similarity index 63% rename from storm-core/src/jvm/org/apache/storm/cluster/Cluster.java rename to storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java index 851858ff6c8..9fd36caf4d1 100644 --- a/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java +++ b/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java @@ -17,6 +17,7 @@ */ 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; @@ -28,7 +29,6 @@ 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; @@ -37,7 +37,7 @@ import java.util.List; import java.util.Map; -public class Cluster { +public class ClusterUtils { public static final String ZK_SEPERATOR = "/"; @@ -55,54 +55,36 @@ public class Cluster { 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 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 Cluster INSTANCE = new Cluster(); - private static Cluster _instance = INSTANCE; + 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 + * 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 + * @param u a Cluster instance */ - public static void setInstance(Cluster u) { + 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. + * 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; @@ -110,8 +92,8 @@ public static void resetInstance() { public static List mkTopoOnlyAcls(Map topoConf) throws NoSuchAlgorithmException { List aclList = null; - String payload = (String)topoConf.get(Config.STORM_ZOOKEEPER_AUTH_PAYLOAD); - if (Utils.isZkAuthenticationConfiguredStormServer(topoConf)){ + 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); @@ -165,11 +147,15 @@ 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 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) throws UnsupportedEncodingException { + public static String lastErrorPath(String stormId, String componentId) { return errorPath(stormId, componentId) + "-last-error"; } @@ -189,32 +175,59 @@ public static String profilerConfigPath(String stormId, String host, Long port, return profilerConfigPath(stormId) + ZK_SEPERATOR + host + "_" + port + "_" + requestType; } - public static T maybeDeserialize(byte[] serialized, Class clazz){ - if (serialized != null){ + public static T maybeDeserialize(byte[] serialized, Class 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 convertExecutorBeats(List executors, ClusterWorkerHeartbeat workerHeartbeat){ + // Ensures that we only return heartbeats for executors assigned to this worker + public static Map convertExecutorBeats(List executors, ClusterWorkerHeartbeat workerHeartbeat) { Map executorWhb = new HashMap<>(); Map executorStatsMap = workerHeartbeat.get_executor_stats(); - for (ExecutorInfo executor : executors){ - if(executorStatsMap.containsKey(executor)){ + for (ExecutorInfo executor : executors) { + if (executorStatsMap.containsKey(executor)) { executorWhb.put(executor, workerHeartbeat); } } return executorWhb; } - public StormClusterState mkStormClusterStateImpl(Object clusterState, List acls, ClusterStateContext context) throws Exception{ - return new StormZkClusterState(clusterState, acls, context); + public StormClusterState mkStormClusterStateImpl(Object StateStorage, List 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 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 acls, ClusterStateContext context) + throws Exception { + return _instance.mkDistributedClusterStateImpl(config, auth_conf, acls, context); } - public static StormClusterState mkStormClusterState(Object clusterState, List acls, ClusterStateContext context) throws Exception{ - return _instance.mkStormClusterStateImpl(clusterState, acls, context); + + public static StormClusterState mkStormClusterState(Object StateStorage, List acls, ClusterStateContext context) throws Exception { + return _instance.mkStormClusterStateImpl(StateStorage, acls, context); } - + // TO be remove public static HashMap> reverseMap(Map map) { HashMap> rtn = new HashMap>(); diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/StateStorage.java similarity index 96% rename from storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java rename to storm-core/src/jvm/org/apache/storm/cluster/StateStorage.java index e76721bd6fc..8895cd1c8a5 100644 --- a/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java +++ b/storm-core/src/jvm/org/apache/storm/cluster/StateStorage.java @@ -22,11 +22,11 @@ import java.util.List; import org.apache.curator.framework.state.ConnectionStateListener; -import org.apache.storm.callback.Callback; +import org.apache.storm.callback.ZKStateChangedCallback; import org.apache.zookeeper.data.ACL; /** - * ClusterState provides the API for the pluggable state store used by the + * 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. @@ -41,7 +41,7 @@ * 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 { +public interface StateStorage { /** * Registers a callback function that gets called when CuratorEvents happen. @@ -50,7 +50,7 @@ public interface ClusterState { * @return is an id that can be passed to unregister(...) to unregister the * callback. */ - String register(Callback callback); + String register(ZKStateChangedCallback callback); /** * Unregisters a callback function that was registered with register(...). @@ -196,8 +196,8 @@ public interface ClusterState { void delete_worker_hb(String path); /** - * Add a ClusterStateListener to the connection. - * @param listener A ClusterStateListener to handle changing cluster state + * Add a StateStorageListener to the connection. + * @param listener A StateStorageListener to handle changing cluster state * events. */ void add_listener(final ConnectionStateListener listener); diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ClusterStateFactory.java b/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java similarity index 90% rename from storm-core/src/jvm/org/apache/storm/cluster/ClusterStateFactory.java rename to storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java index 6474d82ef5e..9803dff16d9 100644 --- a/storm-core/src/jvm/org/apache/storm/cluster/ClusterStateFactory.java +++ b/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java @@ -21,8 +21,8 @@ import java.util.List; import org.apache.zookeeper.data.ACL; -public interface ClusterStateFactory { +public interface StateStorageFactory { - ClusterState mkState(APersistentMap config, APersistentMap auth_conf, List acls, ClusterStateContext context); + StateStorage mkState(APersistentMap config, APersistentMap auth_conf, List acls, ClusterStateContext context); } 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 ede2ba368e3..58b125b3950 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 errors(String stormId, String componentId); diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java similarity index 62% rename from storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java rename to storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java index 3a4205b2664..cd2bc4a936b 100644 --- a/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java +++ b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java @@ -17,36 +17,33 @@ */ package org.apache.storm.cluster; -import clojure.lang.APersistentMap; -import clojure.lang.IFn; -import clojure.lang.PersistentArrayMap; -import clojure.lang.RT; +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.Callback; +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.UnsupportedEncodingException; import java.security.NoSuchAlgorithmException; import java.util.*; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicReference; -public class StormZkClusterState implements StormClusterState { +public class StormClusterStateImpl implements StormClusterState { - private static Logger LOG = LoggerFactory.getLogger(StormZkClusterState.class); + private static Logger LOG = LoggerFactory.getLogger(StormClusterStateImpl.class); - private ClusterState clusterState; + private StateStorage stateStorage; private ConcurrentHashMap assignmentInfoCallback; private ConcurrentHashMap assignmentInfoWithVersionCallback; @@ -64,16 +61,10 @@ public class StormZkClusterState implements StormClusterState { private String stateId; private boolean solo; - public StormZkClusterState(Object clusterState, List acls, ClusterStateContext context) throws Exception { + public StormClusterStateImpl(StateStorage StateStorage, List acls, ClusterStateContext context, boolean solo) 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); - } + this.stateStorage = StateStorage; + this.solo = solo; assignmentInfoCallback = new ConcurrentHashMap<>(); assignmentInfoWithVersionCallback = new ConcurrentHashMap<>(); @@ -86,25 +77,16 @@ public StormZkClusterState(Object clusterState, List acls, ClusterStateCont logConfigCallback = new ConcurrentHashMap<>(); blobstoreCallback = new AtomicReference<>(); - stateId = this.clusterState.register(new Callback() { - - public 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]; + stateId = this.stateStorage.register(new ZKStateChangedCallback() { + public void changed(Watcher.Event.EventType type, String path) { List 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 (root.equals(ClusterUtils.ASSIGNMENTS_ROOT)) { if (size == 1) { // set null and get the old value issueCallback(assignmentsCallback); @@ -114,17 +96,17 @@ public Object execute(T... args) { issueMapCallback(assignmentInfoWithVersionCallback, toks.get(1)); } - } else if (root.equals(Cluster.SUPERVISORS_ROOT)) { + } else if (root.equals(ClusterUtils.SUPERVISORS_ROOT)) { issueCallback(supervisorsCallback); - } else if (root.equals(Cluster.BLOBSTORE_ROOT)) { + } else if (root.equals(ClusterUtils.BLOBSTORE_ROOT)) { issueCallback(blobstoreCallback); - } else if (root.equals(Cluster.STORMS_ROOT) && size > 1) { + } else if (root.equals(ClusterUtils.STORMS_ROOT) && size > 1) { issueMapCallback(stormBaseCallback, toks.get(1)); - } else if (root.equals(Cluster.CREDENTIALS_ROOT) && size > 1) { + } else if (root.equals(ClusterUtils.CREDENTIALS_ROOT) && size > 1) { issueMapCallback(credentialsCallback, toks.get(1)); - } else if (root.equals(Cluster.LOGCONFIG_ROOT) && size > 1) { + } else if (root.equals(ClusterUtils.LOGCONFIG_ROOT) && size > 1) { issueMapCallback(logConfigCallback, toks.get(1)); - } else if (root.equals(Cluster.BACKPRESSURE_ROOT) && size > 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); @@ -133,15 +115,15 @@ public Object execute(T... args) { } - return null; + return; } }); - 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 }; + 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.clusterState.mkdirs(path, acls); + this.stateStorage.mkdirs(path, acls); } } @@ -163,7 +145,7 @@ public List assignments(IFn callback) { if (callback != null) { assignmentsCallback.set(callback); } - return clusterState.get_children(Cluster.ASSIGNMENTS_SUBTREE, callback != null); + return stateStorage.get_children(ClusterUtils.ASSIGNMENTS_SUBTREE, callback != null); } @Override @@ -171,8 +153,8 @@ 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); + byte[] serialized = stateStorage.get_data(ClusterUtils.assignmentPath(stormId), callback != null); + return ClusterUtils.maybeDeserialize(serialized, Assignment.class); } @Override @@ -180,9 +162,13 @@ 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"); + 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; } @@ -192,24 +178,24 @@ 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); + return stateStorage.get_version(ClusterUtils.assignmentPath(stormId), callback != null); } // blobstore state @Override public List blobstoreInfo(String blobKey) { - String path = Cluster.blobstorePath(blobKey); - clusterState.sync_path(path); - return clusterState.get_children(path, false); + String path = ClusterUtils.blobstorePath(blobKey); + stateStorage.sync_path(path); + return stateStorage.get_children(path, false); } @Override public List nimbuses() { List nimbusSummaries = new ArrayList<>(); - List nimbusIds = clusterState.get_children(Cluster.NIMBUSES_SUBTREE, false); + List nimbusIds = stateStorage.get_children(ClusterUtils.NIMBUSES_SUBTREE, false); for (String nimbusId : nimbusIds) { - byte[] serialized = clusterState.get_data(Cluster.nimbusPath(nimbusId), false); - NimbusSummary nimbusSummary = Cluster.maybeDeserialize(serialized, NimbusSummary.class); + byte[] serialized = stateStorage.get_data(ClusterUtils.nimbusPath(nimbusId), false); + NimbusSummary nimbusSummary = ClusterUtils.maybeDeserialize(serialized, NimbusSummary.class); nimbusSummaries.add(nimbusSummary); } return nimbusSummaries; @@ -218,25 +204,25 @@ public List nimbuses() { @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() { + 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"); - clusterState.set_ephemeral_node(Cluster.nimbusPath(nimbusId), Utils.serialize(nimbusSummary), acls); + stateStorage.set_ephemeral_node(ClusterUtils.nimbusPath(nimbusId), Utils.serialize(nimbusSummary), acls); } } }); - clusterState.set_ephemeral_node(Cluster.nimbusPath(nimbusId), Utils.serialize(nimbusSummary), acls); + stateStorage.set_ephemeral_node(ClusterUtils.nimbusPath(nimbusId), Utils.serialize(nimbusSummary), acls); } @Override public List activeStorms() { - return clusterState.get_children(Cluster.STORMS_SUBTREE, false); + return stateStorage.get_children(ClusterUtils.STORMS_SUBTREE, false); } @Override @@ -244,16 +230,14 @@ 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); + 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 = clusterState.get_worker_hb(Cluster.workerbeatPath(stormId, node, port), false); - if (bytes != null) { - return Cluster.maybeDeserialize(bytes, ClusterWorkerHeartbeat.class); - } - return null; + byte[] bytes = stateStorage.get_worker_hb(ClusterUtils.workerbeatPath(stormId, node, port), false); + return ClusterUtils.maybeDeserialize(bytes, ClusterWorkerHeartbeat.class); + } @Override @@ -271,13 +255,13 @@ public List getWorkerProfileRequests(String stormId, NodeInfo no @Override public List getTopologyProfileRequests(String stormId, boolean isThrift) { List profileRequests = new ArrayList<>(); - String path = Cluster.profilerConfigPath(stormId); - if (clusterState.node_exists(path, false)) { - List strs = clusterState.get_children(path, false); + String path = ClusterUtils.profilerConfigPath(stormId); + if (stateStorage.node_exists(path, false)) { + List strs = stateStorage.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); + 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); } @@ -290,8 +274,8 @@ public void setWorkerProfileRequest(String stormId, ProfileRequest profileReques 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); + String path = ClusterUtils.profilerConfigPath(stormId, host, port, profileAction); + stateStorage.set_data(path, Utils.serialize(profileRequest), acls); } @Override @@ -299,8 +283,8 @@ public void deleteTopologyProfileRequests(String stormId, ProfileRequest profile 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); + 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 @@ -311,9 +295,7 @@ public void deleteTopologyProfileRequests(String stormId, ProfileRequest profile public Map executorBeats(String stormId, Map, NodeInfo> executorNodePort) { Map executorWhbs = new HashMap<>(); - LOG.info(executorNodePort.toString()); - Map>> nodePortExecutors = Cluster.reverseMap(executorNodePort); - LOG.info(nodePortExecutors.toString()); + Map>> nodePortExecutors = ClusterUtils.reverseMap(executorNodePort); for (Map.Entry>> entry : nodePortExecutors.entrySet()) { @@ -324,7 +306,8 @@ public Map executorBeats(String stormId, M for (List list : entry.getValue()) { executorInfoList.add(new ExecutorInfo(list.get(0).intValue(), list.get(list.size() - 1).intValue())); } - executorWhbs.putAll(Cluster.convertExecutorBeats(executorInfoList, whb)); + if (whb != null) + executorWhbs.putAll(ClusterUtils.convertExecutorBeats(executorInfoList, whb)); } return executorWhbs; } @@ -334,24 +317,24 @@ public List supervisors(IFn callback) { if (callback != null) { supervisorsCallback.set(callback); } - return clusterState.get_children(Cluster.SUPERVISORS_SUBTREE, callback != null); + return stateStorage.get_children(ClusterUtils.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); + String path = ClusterUtils.supervisorPath(supervisorId); + return ClusterUtils.maybeDeserialize(stateStorage.get_data(path, false), SupervisorInfo.class); } @Override public void setupHeatbeats(String stormId) { - clusterState.mkdirs(Cluster.workerbeatStormRoot(stormId), acls); + stateStorage.mkdirs(ClusterUtils.workerbeatStormRoot(stormId), acls); } @Override public void teardownHeartbeats(String stormId) { try { - clusterState.delete_worker_hb(Cluster.workerbeatStormRoot(stormId)); + stateStorage.delete_worker_hb(ClusterUtils.workerbeatStormRoot(stormId)); } catch (Exception e) { if (Zookeeper.exceptionCause(KeeperException.class, e)) { // do nothing @@ -365,7 +348,7 @@ public void teardownHeartbeats(String stormId) { @Override public void teardownTopologyErrors(String stormId) { try { - clusterState.delete_node(Cluster.errorStormRoot(stormId)); + stateStorage.delete_node(ClusterUtils.errorStormRoot(stormId)); } catch (Exception e) { if (Zookeeper.exceptionCause(KeeperException.class, e)) { // do nothing @@ -378,58 +361,58 @@ public void teardownTopologyErrors(String stormId) { @Override public List heartbeatStorms() { - return clusterState.get_worker_hb_children(Cluster.WORKERBEATS_SUBTREE, false); + return stateStorage.get_worker_hb_children(ClusterUtils.WORKERBEATS_SUBTREE, false); } @Override public List errorTopologies() { - return clusterState.get_children(Cluster.ERRORS_SUBTREE, false); + return stateStorage.get_children(ClusterUtils.ERRORS_SUBTREE, false); } @Override public void setTopologyLogConfig(String stormId, LogConfig logConfig) { - clusterState.set_data(Cluster.logConfigPath(stormId), Utils.serialize(logConfig), acls); + stateStorage.set_data(ClusterUtils.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); + 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 = Cluster.workerbeatPath(stormId, node, port); - clusterState.set_worker_hb(path, Utils.serialize(info), acls); + 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 = Cluster.workerbeatPath(stormId, node, port); - clusterState.delete_worker_hb(path); + String path = ClusterUtils.workerbeatPath(stormId, node, port); + stateStorage.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); + 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 = Cluster.backpressurePath(stormId, node, port); - boolean existed = clusterState.node_exists(path, false); + String path = ClusterUtils.backpressurePath(stormId, node, port); + boolean existed = stateStorage.node_exists(path, false); if (existed) { if (on == false) - clusterState.delete_node(path); + stateStorage.delete_node(path); } else { if (on == true) { - clusterState.set_ephemeral_node(path, null, acls); + stateStorage.set_ephemeral_node(path, null, acls); } } } @@ -440,29 +423,29 @@ public boolean topologyBackpressure(String stormId, IFn callback) { if (callback != null) { backPressureCallback.put(stormId, callback); } - String path = Cluster.backpressureStormRoot(stormId); - List childrens = clusterState.get_children(path, callback != null); + String path = ClusterUtils.backpressureStormRoot(stormId); + List childrens = stateStorage.get_children(path, callback != null); return childrens.size() > 0; } @Override public void setupBackpressure(String stormId) { - clusterState.mkdirs(Cluster.backpressureStormRoot(stormId), acls); + stateStorage.mkdirs(ClusterUtils.backpressureStormRoot(stormId), acls); } @Override public void removeWorkerBackpressure(String stormId, String node, Long port) { - clusterState.delete_node(Cluster.backpressurePath(stormId, node, port)); + stateStorage.delete_node(ClusterUtils.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); + String path = ClusterUtils.stormPath(stormId); + stateStorage.set_data(path, Utils.serialize(stormBase), acls); } - // maybe exit some questions for updateStorm + // To update this function due to APersistentMap/APersistentSet is clojure's structure @Override public void updateStorm(String stormId, StormBase newElems) { @@ -471,9 +454,9 @@ public void updateStorm(String stormId, StormBase newElems) { Map newComponentExecutors = new HashMap<>(); Map componentExecutors = newElems.get_component_executors(); - //componentExecutors maybe be APersistentMap, which don't support put + // componentExecutors maybe be APersistentMap, which don't support "put" for (Map.Entry entry : componentExecutors.entrySet()) { - newComponentExecutors.put(entry.getKey(), entry.getValue()); + newComponentExecutors.put(entry.getKey(), entry.getValue()); } for (Map.Entry entry : stormBase.get_component_executors().entrySet()) { if (!componentExecutors.containsKey(entry.getKey())) { @@ -488,8 +471,9 @@ public void updateStorm(String stormId, StormBase newElems) { Map oldComponentDebug = stormBase.get_component_debug(); Map newComponentDebug = newElems.get_component_debug(); - - Set debugOptionsKeys = oldComponentDebug.keySet(); + /// oldComponentDebug.keySet()/ newComponentDebug.keySet() maybe be APersistentSet, which don't support addAll + Set debugOptionsKeys = new HashSet<>(); + debugOptionsKeys.addAll(oldComponentDebug.keySet()); debugOptionsKeys.addAll(newComponentDebug.keySet()); for (String key : debugOptionsKeys) { boolean enable = false; @@ -511,14 +495,13 @@ public void updateStorm(String stormId, StormBase newElems) { newElems.set_component_debug(ComponentDebug); } - if (StringUtils.isBlank(newElems.get_name())) { newElems.set_name(stormBase.get_name()); } - if (newElems.get_status() == null){ + if (newElems.get_status() == null) { newElems.set_status(stormBase.get_status()); } - if (newElems.get_num_workers() == 0){ + if (newElems.get_num_workers() == 0) { newElems.set_num_workers(stormBase.get_num_workers()); } if (newElems.get_launch_time_secs() == 0) { @@ -533,31 +516,31 @@ public void updateStorm(String stormId, StormBase newElems) { if (newElems.get_status() == null) { newElems.set_status(stormBase.get_status()); } - clusterState.set_data(Cluster.stormPath(stormId), Utils.serialize(newElems), acls); + stateStorage.set_data(ClusterUtils.stormPath(stormId), Utils.serialize(newElems), acls); } @Override public void removeStormBase(String stormId) { - clusterState.delete_node(Cluster.stormPath(stormId)); + stateStorage.delete_node(ClusterUtils.stormPath(stormId)); } @Override public void setAssignment(String stormId, Assignment info) { - clusterState.set_data(Cluster.assignmentPath(stormId), Utils.serialize(info), acls); + stateStorage.set_data(ClusterUtils.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; + String path = ClusterUtils.blobstorePath(key) + ClusterUtils.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); + 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 activeKeys() { - return clusterState.get_children(Cluster.BLOBSTORE_SUBTREE, false); + return stateStorage.get_children(ClusterUtils.BLOBSTORE_SUBTREE, false); } // blobstore state @@ -566,53 +549,53 @@ public List blobstore(IFn callback) { if (callback != null) { blobstoreCallback.set(callback); } - clusterState.sync_path(Cluster.BLOBSTORE_SUBTREE); - return clusterState.get_children(Cluster.BLOBSTORE_SUBTREE, callback != null); + stateStorage.sync_path(ClusterUtils.BLOBSTORE_SUBTREE); + return stateStorage.get_children(ClusterUtils.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)); + 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); - clusterState.delete_node(Cluster.blobstorePath(blobKey)); + stateStorage.delete_node(ClusterUtils.blobstorePath(blobKey)); } @Override public void removeKeyVersion(String blobKey) { - clusterState.delete_node(Cluster.blobstoreMaxKeySequenceNumberPath(blobKey)); + stateStorage.delete_node(ClusterUtils.blobstoreMaxKeySequenceNumberPath(blobKey)); } @Override - 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) { - 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 childrens = clusterState.get_children(path, false); - - Collections.sort(childrens); - - while (childrens.size() >= 10) { - clusterState.delete_node(path + Cluster.ZK_SEPERATOR + childrens.remove(0)); + 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 childrens = stateStorage.get_children(path, false); + + Collections.sort(childrens, new Comparator() { + public int compare(String arg0, String arg1) { + return Long.compare(Long.parseLong(arg0.substring(1)), Long.parseLong(arg1.substring(1))); } - } catch (UnsupportedEncodingException e) { - throw Utils.wrapInRuntime(e); + }); + + while (childrens.size() > 10) { + stateStorage.delete_node(path + ClusterUtils.ZK_SEPERATOR + childrens.remove(0)); } } @@ -620,19 +603,19 @@ public void reportError(String stormId, String componentId, String node, Integer public List errors(String stormId, String componentId) { List errorInfos = new ArrayList<>(); try { - String path = Cluster.errorPath(stormId, componentId); - if (clusterState.node_exists(path, false)) { - List childrens = clusterState.get_children(path, false); + String path = ClusterUtils.errorPath(stormId, componentId); + if (stateStorage.node_exists(path, false)) { + List childrens = stateStorage.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); + 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() { public int compare(ErrorInfo arg0, ErrorInfo arg1) { - return Integer.compare(arg0.get_error_time_secs(), arg1.get_error_time_secs()); + return -Integer.compare(arg0.get_error_time_secs(), arg1.get_error_time_secs()); } }); } catch (Exception e) { @@ -644,23 +627,21 @@ public int compare(ErrorInfo arg0, ErrorInfo arg1) { @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); + + 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 aclList = Cluster.mkTopoOnlyAcls(topoConf); - String path = Cluster.credentialsPath(stormId); - clusterState.set_data(path, Utils.serialize(creds), aclList); + List aclList = ClusterUtils.mkTopoOnlyAcls(topoConf); + String path = ClusterUtils.credentialsPath(stormId); + stateStorage.set_data(path, Utils.serialize(creds), aclList); } @@ -669,15 +650,15 @@ 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); + String path = ClusterUtils.credentialsPath(stormId); + return ClusterUtils.maybeDeserialize(stateStorage.get_data(path, callback != null), Credentials.class); } @Override public void disconnect() { - clusterState.unregister(stateId); + stateStorage.unregister(stateId); if (solo) - clusterState.close(); + stateStorage.close(); } } diff --git a/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java similarity index 85% rename from storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java rename to storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java index 1bd534e9cda..8ac0adcc260 100644 --- a/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java +++ b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java @@ -22,8 +22,9 @@ 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.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; @@ -40,11 +41,11 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; -public class DistributedClusterState implements ClusterState { +public class ZKStateStorage implements StateStorage { - private static Logger LOG = LoggerFactory.getLogger(DistributedClusterState.class); + private static Logger LOG = LoggerFactory.getLogger(ZKStateStorage.class); - private ConcurrentHashMap callbacks = new ConcurrentHashMap(); + private ConcurrentHashMap callbacks = new ConcurrentHashMap(); private CuratorFramework zkWriter; private CuratorFramework zkReader; private AtomicBoolean active; @@ -53,10 +54,11 @@ public class DistributedClusterState implements ClusterState { private Map authConf; private Map conf; - public DistributedClusterState(Map conf, Map authConf, List acls, ClusterStateContext context) throws Exception { + public ZKStateStorage(Map conf, Map authConf, List 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(); @@ -76,9 +78,9 @@ public void execute(Watcher.Event.KeeperState state, Watcher.Event.EventType typ } if (!type.equals(Watcher.Event.EventType.None)) { - for (Map.Entry e : callbacks.entrySet()) { - Callback fn = e.getValue(); - fn.execute(type, path); + for (Map.Entry e : callbacks.entrySet()) { + ZKStateChangedCallback fn = e.getValue(); + fn.changed(type, path); } } } @@ -92,13 +94,13 @@ public void execute(Watcher.Event.KeeperState state, Watcher.Event.EventType typ 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); + LOG.debug("Received event {} : {} : {}", state, type, path); } if (!type.equals(Watcher.Event.EventType.None)) { - for (Map.Entry e : callbacks.entrySet()) { - Callback fn = e.getValue(); - fn.execute(type, path); + for (Map.Entry e : callbacks.entrySet()) { + ZKStateChangedCallback fn = e.getValue(); + fn.changed(type, path); } } } @@ -112,7 +114,8 @@ public void execute(Watcher.Event.KeeperState state, Watcher.Event.EventType typ @SuppressWarnings("unchecked") private CuratorFramework mkZk() throws IOException { - return Zookeeper.mkClient(conf, (List) conf.get(Config.STORM_ZOOKEEPER_SERVERS), conf.get(Config.STORM_ZOOKEEPER_PORT), "", authConf); + return Zookeeper.mkClient(conf, (List) conf.get(Config.STORM_ZOOKEEPER_SERVERS), conf.get(Config.STORM_ZOOKEEPER_PORT), "", + new DefaultWatcherCallBack(), authConf); } @SuppressWarnings("unchecked") @@ -127,9 +130,9 @@ public void delete_node_blobstore(String path, String nimbusHostPortInfo) { } @Override - public String register( Callback callback) { + public String register(ZKStateChangedCallback callback) { String id = UUID.randomUUID().toString(); - this.callbacks.put(id,callback); + this.callbacks.put(id, callback); return id; } @@ -159,11 +162,11 @@ public void set_ephemeral_node(String path, byte[] data, List acls) { if (Zookeeper.exists(zkWriter, path, false)) { try { Zookeeper.setData(zkWriter, path, data); - } catch (RuntimeException e) { - if (Utils.exceptionCauseIsInstanceOf(KeeperException.NodeExistsException.class, e)) { + } catch (Exception e) { + if (Utils.exceptionCauseIsInstanceOf(KeeperException.NoNodeException.class, e)) { Zookeeper.createNode(zkWriter, path, data, CreateMode.EPHEMERAL, acls); } else { - throw e; + throw Utils.wrapInRuntime(e); } } diff --git a/storm-core/test/jvm/org/apache/storm/ClusterTest.java b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java similarity index 59% rename from storm-core/test/jvm/org/apache/storm/ClusterTest.java rename to storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java index ef43afeaece..19b04f28ac6 100644 --- a/storm-core/test/jvm/org/apache/storm/ClusterTest.java +++ b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java @@ -15,8 +15,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.storm; +package org.apache.storm.cluster; +import clojure.lang.APersistentMap; +import org.apache.storm.utils.Utils; +import org.apache.zookeeper.data.ACL; -public class ClusterTest { +import java.util.List; + +public class ZKStateStorageFactory implements StateStorageFactory{ + + @Override + public StateStorage mkState(APersistentMap config, APersistentMap auth_conf, List acls, ClusterStateContext context) { + try { + return new ZKStateStorage(config, auth_conf, acls, context); + }catch (Exception e){ + throw Utils.wrapInRuntime(e); + } + } } 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 5d67a545142..2f1440c88eb 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(); } } 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 f1c7f323706..c28051547de 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 static void resetInstance() { _instance = INSTANCE; } - public static CuratorFramework mkClient(Map conf, List servers, Object port, String root) { - return mkClient(conf, servers, port, root, new DefaultWatcherCallBack()); + public CuratorFramework mkClientImpl(Map conf, List servers, Object port, String root) { + return mkClientImpl(conf, servers, port, root, new DefaultWatcherCallBack()); } - public static CuratorFramework mkClient(Map conf, List servers, Object port, Map authConf) { - return mkClient(conf, servers, port, "", new DefaultWatcherCallBack(), authConf); + public CuratorFramework mkClientImpl(Map conf, List servers, Object port, Map authConf) { + return mkClientImpl(conf, servers, port, "", new DefaultWatcherCallBack(), authConf); } - public static CuratorFramework mkClient(Map conf, List servers, Object port, String root, Map authConf) { - return mkClient(conf, servers, port, root, new DefaultWatcherCallBack(), authConf); + public CuratorFramework mkClientImpl(Map conf, List servers, Object port, String root, Map authConf) { + return mkClientImpl(conf, servers, port, root, new DefaultWatcherCallBack(), authConf); } public static CuratorFramework mkClient(Map conf, List 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 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 void eventReceived(CuratorFramework _fk, CuratorEvent e) throws Exception * * @return */ - public static CuratorFramework mkClient(Map conf, List servers, Object port, String root, final WatcherCallBack watcher) { - return mkClient(conf, servers, port, root, watcher, null); + public CuratorFramework mkClientImpl(Map conf, List 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 acls) { @@ -347,7 +351,7 @@ public static ILeaderElector zkLeaderElector(Map conf) throws UnknownHostExcepti protected ILeaderElector zkLeaderElectorImpl(Map conf) throws UnknownHostException { List servers = (List) 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 leaderLatchAtomicReference = new AtomicReference<>(new LeaderLatch(zk, leaderLockPath, id)); 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 d374511019b..d4fab3f72f4 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")))) 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 d0b988217ca..fa34355f1b9 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 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 d4402fb04bd..772a2323e8e 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) 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 c98a68bcf78..b89b7bbfe95 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) From 2ee8bec8458b02bca6af757ce0f1052a16c660b8 Mon Sep 17 00:00:00 2001 From: "xiaojian.fxj" Date: Mon, 15 Feb 2016 09:37:32 +0800 Subject: [PATCH 05/10] port pacemaker_state_factory.clj --- .../org/apache/storm/command/heartbeats.clj | 2 +- .../clj/org/apache/storm/daemon/worker.clj | 2 +- .../pacemaker/pacemaker_state_factory.clj | 6 +- .../src/clj/org/apache/storm/testing.clj | 2 +- .../apache/storm/cluster/ClusterUtils.java | 20 +- .../{StateStorage.java => IStateStorage.java} | 3 +- ...sterState.java => IStormClusterState.java} | 2 +- .../storm/cluster/PaceMakerStateStorage.java | 212 ++++++++++++++++++ .../cluster/PaceMakerStateStorageFactory.java | 64 ++++++ .../storm/cluster/StateStorageFactory.java | 2 +- .../storm/cluster/StormClusterStateImpl.java | 8 +- .../apache/storm/cluster/ZKStateStorage.java | 4 +- .../storm/cluster/ZKStateStorageFactory.java | 2 +- .../storm/pacemaker/PacemakerClient.java | 1 - .../MockedPaceMakerStateStorageFactory.java | 32 +++ .../clj/org/apache/storm/cluster_test.clj | 10 +- .../storm/pacemaker_state_factory_test.clj | 57 ++--- 17 files changed, 369 insertions(+), 60 deletions(-) rename storm-core/src/jvm/org/apache/storm/cluster/{StateStorage.java => IStateStorage.java} (99%) rename storm-core/src/jvm/org/apache/storm/cluster/{StormClusterState.java => IStormClusterState.java} (99%) create mode 100644 storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java create mode 100644 storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java create mode 100644 storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedPaceMakerStateStorageFactory.java 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 af86b699415..c4413f0f22c 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" 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 a79300957a2..ae5be5740e4 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) 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 28f792d3c4f..a36da3ad152 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)) diff --git a/storm-core/src/clj/org/apache/storm/testing.clj b/storm-core/src/clj/org/apache/storm/testing.clj index 470a14f49b4..5a0bdf2af5a 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) 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 9fd36caf4d1..b30d1d2e5ba 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 static Map convertExecutorBeats(Lis return executorWhb; } - public StormClusterState mkStormClusterStateImpl(Object StateStorage, List acls, ClusterStateContext context) throws Exception { - if (StateStorage instanceof StateStorage) { - return new StormClusterStateImpl((StateStorage) StateStorage, acls, context, false); + public IStormClusterState mkStormClusterStateImpl(Object stateStorage, List 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 acls, ClusterStateContext context) + public IStateStorage mkStateStorageImpl(APersistentMap config, APersistentMap auth_conf, List 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 StateStorage mkDistributedClusterStateImpl(APersistentMap config, APersis } 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 acls, ClusterStateContext context) + public static IStateStorage mkStateStorage(APersistentMap config, APersistentMap auth_conf, List 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 acls, ClusterStateContext context) throws Exception { + public static IStormClusterState mkStormClusterState(Object StateStorage, List acls, ClusterStateContext context) throws Exception { return _instance.mkStormClusterStateImpl(StateStorage, acls, context); } diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StateStorage.java b/storm-core/src/jvm/org/apache/storm/cluster/IStateStorage.java similarity index 99% rename from storm-core/src/jvm/org/apache/storm/cluster/StateStorage.java rename to storm-core/src/jvm/org/apache/storm/cluster/IStateStorage.java index 8895cd1c8a5..1a2b14f2b22 100644 --- a/storm-core/src/jvm/org/apache/storm/cluster/StateStorage.java +++ b/storm-core/src/jvm/org/apache/storm/cluster/IStateStorage.java @@ -18,7 +18,6 @@ package org.apache.storm.cluster; import clojure.lang.APersistentMap; -import clojure.lang.IFn; import java.util.List; import org.apache.curator.framework.state.ConnectionStateListener; @@ -41,7 +40,7 @@ * 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 { +public interface IStateStorage { /** * Registers a callback function that gets called when CuratorEvents happen. diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java similarity index 99% rename from storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java rename to storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java index 58b125b3950..59d1af724a7 100644 --- a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java +++ b/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java @@ -26,7 +26,7 @@ import java.util.List; import java.util.Map; -public interface StormClusterState { +public interface IStormClusterState { public List assignments(IFn callback); public Assignment assignmentInfo(String stormId, IFn callback); 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 00000000000..1226c55b1b5 --- /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 acls) { + return stateStorage.create_sequential(path, data, acls); + } + + @Override + public void mkdirs(String path, List 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 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 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 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 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 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); + } +} 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 00000000000..eafd2e73ea7 --- /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 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 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 acls, ClusterStateContext context) throws Exception { + return ClusterUtils.mkStateStorage(config, auth_conf, acls, context); + } + + public PacemakerClient initMakeClientImpl(APersistentMap config) { + return new PacemakerClient(config); + } +} 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 9803dff16d9..c2477d67210 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 @@ public interface StateStorageFactory { - StateStorage mkState(APersistentMap config, APersistentMap auth_conf, List acls, ClusterStateContext context); + IStateStorage mkStore(APersistentMap config, APersistentMap auth_conf, List acls, ClusterStateContext context); } 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 cd2bc4a936b..8df5885eab9 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.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 assignmentInfoCallback; private ConcurrentHashMap assignmentInfoWithVersionCallback; @@ -61,7 +61,7 @@ public class StormClusterStateImpl implements StormClusterState { private String stateId; private boolean solo; - public StormClusterStateImpl(StateStorage StateStorage, List acls, ClusterStateContext context, boolean solo) throws Exception { + public StormClusterStateImpl(IStateStorage StateStorage, List acls, ClusterStateContext context, boolean solo) throws Exception { this.stateStorage = StateStorage; this.solo = solo; @@ -615,7 +615,7 @@ public List errors(String stormId, String componentId) { } Collections.sort(errorInfos, new Comparator() { 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) { 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 8ac0adcc260..b277751b954 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.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 @@ private CuratorFramework mkZk(WatcherCallBack watcher) throws NumberFormatExcept @Override public void delete_node_blobstore(String path, String nimbusHostPortInfo) { - + Zookeeper.deleteNodeBlobstore(zkWriter, path, nimbusHostPortInfo); } @Override 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 19b04f28ac6..f3b9253ad16 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 @@ public class ZKStateStorageFactory implements StateStorageFactory{ @Override - public StateStorage mkState(APersistentMap config, APersistentMap auth_conf, List acls, ClusterStateContext context) { + public IStateStorage mkStore(APersistentMap config, APersistentMap auth_conf, List acls, ClusterStateContext context) { try { return new ZKStateStorage(config, auth_conf, acls, context); }catch (Exception e){ 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 34f36653341..af0e8f3c52a 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 String name() { public String secretKey() { return secret; } - public HBMessage send(HBMessage m) { waitUntilReady(); LOG.debug("Sending message: {}", m.toString()); 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 00000000000..0253afb662d --- /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(); + } +} 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 fa34355f1b9..39adb9e2875 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 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 09252372d1c..1a7bd2cd87b 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)))))) From e0f3cb5f322c1dc09d05803e93d1fb3e6a3baff0 Mon Sep 17 00:00:00 2001 From: "xiaojian.fxj" Date: Tue, 16 Feb 2016 11:40:48 +0800 Subject: [PATCH 06/10] fix nimbus_test.clj --- storm-core/test/clj/org/apache/storm/nimbus_test.clj | 2 -- 1 file changed, 2 deletions(-) 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 2a65efc3947..09c4371e5bc 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)] From afcd0c6c53aca7f99d39f10b91a7b45fda424fe5 Mon Sep 17 00:00:00 2001 From: "xiaojian.fxj" Date: Fri, 19 Feb 2016 15:21:48 +0800 Subject: [PATCH 07/10] removed any clojure internals --- .../src/clj/org/apache/storm/converter.clj | 9 +++ .../clj/org/apache/storm/daemon/nimbus.clj | 4 +- .../org/apache/storm/daemon/supervisor.clj | 8 +- .../clj/org/apache/storm/daemon/worker.clj | 6 +- .../src/clj/org/apache/storm/testing.clj | 2 +- .../apache/storm/cluster/ClusterUtils.java | 41 +++------- .../apache/storm/cluster/ExecutorBeat.java | 44 +++++++++++ .../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(-) create mode 100644 storm-core/src/jvm/org/apache/storm/cluster/ExecutorBeat.java diff --git a/storm-core/src/clj/org/apache/storm/converter.clj b/storm-core/src/clj/org/apache/storm/converter.clj index 18647b1f4ac..c845cd4951b 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.) 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 6bdbdc0d5da..beb66390d4a 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) 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 c1f058f3c4a..58f6291d4d9 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)) 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 b80cd9edb6b..395be233c6b 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 diff --git a/storm-core/src/clj/org/apache/storm/testing.clj b/storm-core/src/clj/org/apache/storm/testing.clj index eef7754cec9..3dee54b1eb8 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))) 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 0c663f062b7..aae4231e300 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 static T maybeDeserialize(byte[] serialized, Class clazz) { * @param workerHeartbeat * @return */ - public static Map convertExecutorBeats(List executors, ClusterWorkerHeartbeat workerHeartbeat) { - Map executorWhb = new HashMap<>(); + public static Map convertExecutorBeats(List executors, ClusterWorkerHeartbeat workerHeartbeat) { + Map executorWhb = new HashMap<>(); Map 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 IStormClusterState mkStormClusterStateImpl(Object stateStorage, List 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 acls, ClusterStateContext context) throws Exception { + public IStateStorage mkStateStorageImpl(Map config, Map auth_conf, List 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 IStateStorage mkStateStorageImpl(APersistentMap config, APersistentMap au return stateStorage; } - public static IStateStorage mkStateStorage(APersistentMap config, APersistentMap auth_conf, List acls, ClusterStateContext context) throws Exception { + public static IStateStorage mkStateStorage(Map config, Map auth_conf, List acls, ClusterStateContext context) throws Exception { return _instance.mkStateStorageImpl(config, auth_conf, acls, context); } @@ -238,26 +236,7 @@ public static IStormClusterState mkStormClusterState(Object StateStorage, List HashMap> reverseMap(Map map) { - HashMap> rtn = new HashMap>(); - if (map == null) { - return rtn; - } - for (Map.Entry entry : map.entrySet()) { - K key = entry.getKey(); - V val = entry.getValue(); - List list = rtn.get(val); - if (list == null) { - list = new ArrayList(); - 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; 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 00000000000..b32615e188a --- /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; + } +} 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 1a2b14f2b22..0b6f043f295 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 @@ */ 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. 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 01cf56a19d5..c88935e74cd 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.Map; public interface IStormClusterState { - public List assignments(IFn callback); + public List 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 blobstoreInfo(String blobKey); @@ -43,7 +41,7 @@ public interface IStormClusterState { public List 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 executorBeats(String stormId, Map, NodeInfo> executorNodePort); + public Map executorBeats(String stormId, Map, NodeInfo> executorNodePort); - public List supervisors(IFn callback); + public List 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 activeKeys(); - public List blobstore(IFn callback); + public List 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(); 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 a9c4d89312c..c29078effa5 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.LoggerFactory; import java.util.List; +import java.util.Map; public class PaceMakerStateStorage implements IStateStorage { @@ -104,7 +104,7 @@ public byte[] get_data(String path, boolean watch) { } @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); } 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 eafd2e73ea7..3111e04942c 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 static void resetInstance() { } @Override - public IStateStorage mkStore(APersistentMap config, APersistentMap auth_conf, List acls, ClusterStateContext context) { + public IStateStorage mkStore(Map config, Map auth_conf, List acls, ClusterStateContext context) { try { return new PaceMakerStateStorage(initMakeClient(config), initZKstate(config, auth_conf, acls, context)); } catch (Exception e) { @@ -46,19 +46,19 @@ public IStateStorage mkStore(APersistentMap config, APersistentMap auth_conf, Li } } - public static IStateStorage initZKstate(APersistentMap config, APersistentMap auth_conf, List acls, ClusterStateContext context) throws Exception { + public static IStateStorage initZKstate(Map config, Map auth_conf, List 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 acls, ClusterStateContext context) throws Exception { + public IStateStorage initZKstateImpl(Map config, Map auth_conf, List 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); } } 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 110da41ba27..0929750f1d2 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 acls, ClusterStateContext context); - + IStateStorage mkStore(Map config, Map auth_conf, List acls, ClusterStateContext context); } 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 17c864175c1..5fa586a57d9 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.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 assignmentInfoCallback; - private ConcurrentHashMap assignmentInfoWithVersionCallback; - private ConcurrentHashMap assignmentVersionCallback; - private AtomicReference supervisorsCallback; + private ConcurrentHashMap assignmentInfoCallback; + private ConcurrentHashMap assignmentInfoWithVersionCallback; + private ConcurrentHashMap assignmentVersionCallback; + private AtomicReference supervisorsCallback; // we want to reigister a topo directory getChildren callback for all workers of this dir - private ConcurrentHashMap backPressureCallback; - private AtomicReference assignmentsCallback; - private ConcurrentHashMap stormBaseCallback; - private AtomicReference blobstoreCallback; - private ConcurrentHashMap credentialsCallback; - private ConcurrentHashMap logConfigCallback; + private ConcurrentHashMap backPressureCallback; + private AtomicReference assignmentsCallback; + private ConcurrentHashMap stormBaseCallback; + private AtomicReference blobstoreCallback; + private ConcurrentHashMap credentialsCallback; + private ConcurrentHashMap logConfigCallback; private List acls; private String stateId; @@ -129,20 +127,20 @@ public void changed(Watcher.Event.EventType type, String path) { } - protected void issueCallback(AtomicReference cb) { - IFn callback = cb.getAndSet(null); + protected void issueCallback(AtomicReference cb) { + Runnable callback = cb.getAndSet(null); if (callback != null) - callback.invoke(); + callback.run(); } - protected void issueMapCallback(ConcurrentHashMap callbackConcurrentHashMap, String key) { - IFn callback = callbackConcurrentHashMap.remove(key); + protected void issueMapCallback(ConcurrentHashMap callbackConcurrentHashMap, String key) { + Runnable callback = callbackConcurrentHashMap.remove(key); if (callback != null) - callback.invoke(); + callback.run(); } @Override - public List assignments(IFn callback) { + public List assignments(Runnable callback) { if (callback != null) { assignmentsCallback.set(callback); } @@ -150,7 +148,7 @@ public List assignments(IFn callback) { } @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 Assignment assignmentInfo(String stormId, IFn callback) { } @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 List activeStorms() { } @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 void deleteTopologyProfileRequests(String stormId, ProfileRequest profile * @return */ @Override - public Map executorBeats(String stormId, Map, NodeInfo> executorNodePort) { - Map executorWhbs = new HashMap<>(); + public Map executorBeats(String stormId, Map, NodeInfo> executorNodePort) { + Map executorWhbs = new HashMap<>(); - Map>> nodePortExecutors = ClusterUtils.reverseMap(executorNodePort); + Map>> nodePortExecutors = Utils.reverseMap(executorNodePort); for (Map.Entry>> entry : nodePortExecutors.entrySet()) { @@ -319,7 +319,7 @@ public Map executorBeats(String stormId, Map supervisors(IFn callback) { + public List supervisors(Runnable callback) { if (callback != null) { supervisorsCallback.set(callback); } @@ -342,7 +342,7 @@ public void teardownHeartbeats(String stormId) { 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 void teardownTopologyErrors(String stormId) { 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 void setTopologyLogConfig(String stormId, LogConfig logConfig) { } @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 void workerBackpressure(String stormId, String node, Long port, boolean o * @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 List activeKeys() { // blobstore state @Override - public List blobstore(IFn callback) { + public List blobstore(Runnable callback) { if (callback != null) { blobstoreCallback.set(callback); } @@ -602,7 +602,7 @@ public void reportError(String stormId, String componentId, String node, Long po 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 void setCredentials(String stormId, Credentials creds, Map topoConf) thro } @Override - public Credentials credentials(String stormId, IFn callback) { + public Credentials credentials(String stormId, Runnable callback) { if (callback != null) { credentialsCallback.put(stormId, callback); } 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 b277751b954..56115ce01fc 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 byte[] get_data(String path, boolean watch) { } @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); } 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 956c20ee3e4..232488b13eb 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 acls, ClusterStateContext context) { + public IStateStorage mkStore(Map config, Map auth_conf, List acls, ClusterStateContext context) { try { return new ZKStateStorage(config, auth_conf, acls, context); } catch (Exception e) { 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 f80b0a4bf8e..e5b2666eea4 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.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.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 static void deleteNode(CuratorFramework zk, String path){ 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 void mkdirsImpl(CuratorFramework zk, String path, List acls) { 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 static byte[] getData(CuratorFramework zk, String path, boolean watch){ } } } 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 static List mkInprocessZookeeper(String localdir, Integer port) throws Ex } 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 @@ protected ILeaderElector zkLeaderElectorImpl(Map conf) throws UnknownHostExcepti 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 static APersistentMap getDataWithVersion(CuratorFramework zk, String path } 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 static String normalizePath(String path) { 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; - } - } From b7bc9bf9bd0793f2acbf158651e3200b30f97d99 Mon Sep 17 00:00:00 2001 From: "xiaojian.fxj" Date: Sun, 21 Feb 2016 10:30:05 +0800 Subject: [PATCH 08/10] resolve my little fault --- storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 aae4231e300..1095fff99c2 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 static void resetInstance() { public static List mkTopoOnlyAcls(Map topoConf) throws NoSuchAlgorithmException { List 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); From d70952513336532c80957e5398bfbedacff78cb7 Mon Sep 17 00:00:00 2001 From: "xiaojian.fxj" Date: Tue, 23 Feb 2016 22:05:50 +0800 Subject: [PATCH 09/10] convert ExecutorStats to stats of clojure by clojurify-executor-stats --- 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(-) diff --git a/storm-core/src/clj/org/apache/storm/converter.clj b/storm-core/src/clj/org/apache/storm/converter.clj index c845cd4951b..e269c5d519a 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) } 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 e43bab93589..2f6587afe41 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 From b09e2755c2fedcb9048afda63bea897fe1dbd12b Mon Sep 17 00:00:00 2001 From: "xiaojian.fxj" Date: Wed, 24 Feb 2016 11:19:06 +0800 Subject: [PATCH 10/10] update/fix some codes based on @revans2 --- .../clj/org/apache/storm/daemon/nimbus.clj | 2 +- .../org/apache/storm/daemon/supervisor.clj | 4 +- .../clj/org/apache/storm/daemon/worker.clj | 8 +- .../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(-) 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 2f6587afe41..e524ec25e6d 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 %))) 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 0cee4148920..1446ac98195 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) 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 af88f6a875f..110d415c36e 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)] 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 1095fff99c2..96c177bc397 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 IStormClusterState mkStormClusterStateImpl(Object stateStorage, List 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 acls, ClusterStateContext context) throws Exception { @@ -237,25 +236,9 @@ public static IStormClusterState mkStormClusterState(Object StateStorage, List getWorkerProfileRequests(String stormId, NodeInfo nodeInfo, boolean isThrift); + public List getWorkerProfileRequests(String stormId, NodeInfo nodeInfo); - public List getTopologyProfileRequests(String stormId, boolean isThrift); + public List getTopologyProfileRequests(String stormId); public void setWorkerProfileRequest(String stormId, ProfileRequest profileRequest); 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 c29078effa5..c42bd389f0c 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 void set_worker_hb(String path, byte[] data, List acls) { 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 byte[] get_worker_hb(String path, boolean watch) { 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 List get_worker_hb_children(String path, boolean watch) { 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 void delete_worker_hb(String path) { 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); } } } 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 5fa586a57d9..bde767039cc 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 void changed(Watcher.Event.EventType type, String path) { } 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 ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, String node, Lo } @Override - public List getWorkerProfileRequests(String stormId, NodeInfo nodeInfo, boolean isThrift) { + public List getWorkerProfileRequests(String stormId, NodeInfo nodeInfo) { List requests = new ArrayList<>(); - List profileRequests = getTopologyProfileRequests(stormId, isThrift); + List profileRequests = getTopologyProfileRequests(stormId); for (ProfileRequest profileRequest : profileRequests) { NodeInfo nodeInfo1 = profileRequest.get_nodeInfo(); if (nodeInfo1.equals(nodeInfo)) @@ -254,7 +254,7 @@ public List getWorkerProfileRequests(String stormId, NodeInfo no } @Override - public List getTopologyProfileRequests(String stormId, boolean isThrift) { + public List getTopologyProfileRequests(String stormId) { List profileRequests = new ArrayList<>(); String path = ClusterUtils.profilerConfigPath(stormId); if (stateStorage.node_exists(path, false)) { @@ -382,6 +382,9 @@ public void setTopologyLogConfig(String stormId, LogConfig logConfig) { @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 int compare(String arg0, String arg1) { @Override public List errors(String stormId, String componentId) { List errorInfos = new ArrayList<>(); - try { - String path = ClusterUtils.errorPath(stormId, componentId); - if (stateStorage.node_exists(path, false)) { - List 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 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() { - 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() { + public int compare(ErrorInfo arg0, ErrorInfo arg1) { + return Integer.compare(arg1.get_error_time_secs(), arg0.get_error_time_secs()); + } + }); return errorInfos; } 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 56115ce01fc..4cf0c054191 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 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 e : callbacks.entrySet()) { + ZKStateChangedCallback fn = e.getValue(); + fn.changed(type, path); + } + } + } + } + } + public ZKStateStorage(Map conf, Map authConf, List acls, ClusterStateContext context) throws Exception { this.conf = conf; this.authConf = authConf; @@ -66,45 +86,9 @@ public ZKStateStorage(Map conf, Map authConf, List acls, Cl 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 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 e : callbacks.entrySet()) { - ZKStateChangedCallback fn = e.getValue(); - fn.changed(type, path); - } - } - } - } - }); + zkReader = mkZk(new ZkWatcherCallBack()); } else { zkReader = zkWriter; } @@ -157,15 +141,15 @@ public void delete_node(String path) { @Override public void set_ephemeral_node(String path, byte[] data, List 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 Integer get_version(String path, boolean watch) throws Exception { @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 void set_data(String path, byte[] data, List acls) { 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 void stateChanged(CuratorFramework curatorFramework, ConnectionState conn 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 toks = Zookeeper.tokenizePath(path); - int size = toks.size(); - if (size > 0) { - toks.remove(size - 1); - } - return Zookeeper.toksToPath(toks); - } } 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 e5b2666eea4..5e9039a32ec 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 static List tokenizePath(String path) { } public static String parentPath(String path) { - List tokens = tokenizePath(path); - tokens.remove(tokens.size() - 1); - return "/" + StringUtils.join(tokens, "/"); + List toks = Zookeeper.tokenizePath(path); + int size = toks.size(); + if (size > 0) { + toks.remove(size - 1); + } + return Zookeeper.toksToPath(toks); } public static String toksToPath(List toks) {