/**
* 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 com.alibaba.jstorm.zk;
import java.util.List;
import java.util.Map;
import com.alibaba.jstorm.cluster.Cluster;
import backtype.storm.Config;
import backtype.storm.utils.Utils;
import com.alibaba.jstorm.cluster.ClusterState;
import com.alibaba.jstorm.cluster.DistributedClusterState;
import com.google.common.collect.Maps;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class ZkTool {
private static final Logger LOG = LoggerFactory.getLogger(ZkTool.class);
public static final String READ_CMD = "read";
public static final String RM_CMD = "rm";
public static final String LIST_CMD = "list";
public static final String CLEAN_CMD = "clean";
public static void usage() {
System.out.println("Read ZK node's data, please do as following:");
System.out.println(ZkTool.class.getName() + " read zkpath");
System.out.println("\nDelete topology backup assignment, please do as following:");
System.out.println(ZkTool.class.getName() + " rm topologyname");
System.out.println("\nlist subdirectory of zkPath , please do as following:");
System.out.println(ZkTool.class.getName() + " list zkpath");
System.out.println("\nDelete all nodes about a topologyId of zk , please do as following:");
System.out.println(ZkTool.class.getName() + " clean topologyId");
}
public static String getData(DistributedClusterState zkClusterState,
String path) throws Exception {
byte[] data = zkClusterState.get_data(path, false);
if (data == null || data.length == 0) {
return null;
}
Object obj = Utils.deserialize(data, null);
return obj.toString();
}
public static void list(String path) {
DistributedClusterState zkClusterState = null;
try {
conf.put(Config.STORM_ZOOKEEPER_ROOT, "/");
zkClusterState = new DistributedClusterState(conf);
List<String> children = zkClusterState.get_children(path, false);
if (children == null || children.isEmpty()) {
System.out.println("No children of " + path);
} else {
StringBuilder sb = new StringBuilder();
sb.append("Zk node children of " + path + "\n");
for (String str : children) {
sb.append(" " + str + ",");
}
sb.append("\n");
System.out.println(sb.toString());
}
} catch (Exception e) {
if (zkClusterState == null) {
LOG.error("Failed to connect ZK ", e);
} else {
LOG.error("Failed to list children of " + path + "\n", e);
}
} finally {
if (zkClusterState != null) {
zkClusterState.close();
}
}
}
/**
* Note: this method can't delete zkCache right now because of
* new DistributedClusterState(conf)
*/
public static void cleanTopology(String topologyId) {
DistributedClusterState zkClusterState = null;
try {
zkClusterState = new DistributedClusterState(conf);
String rootDir = String.valueOf(conf.get(Config.STORM_ZOOKEEPER_ROOT));
String assignmentPath = "/assignments/" + topologyId;
String stormBase = "/topology/" + topologyId;
String taskBeats = "/taskbeats/" + topologyId;
String tasks = "/tasks/" + topologyId;
String taskErrors = "/taskerrors/" + topologyId;
String monitor = "/monitor/" + topologyId;
if (zkClusterState.node_existed(assignmentPath, false)) {
try {
zkClusterState.delete_node(assignmentPath);
} catch (Exception e) {
LOG.error("Could not remove assignments for " + topologyId, e);
}
} else {
System.out.println(" node of " + rootDir + assignmentPath + " isn't existed ");
}
if (zkClusterState.node_existed(stormBase, false)) {
try {
zkClusterState.delete_node(stormBase);
} catch (Exception e) {
LOG.error("Failed to remove storm base for " + topologyId, e);
}
} else {
System.out.println(" node of " + rootDir + stormBase + " isn't existed ");
}
if (zkClusterState.node_existed(taskBeats, false)) {
try {
zkClusterState.delete_node(taskBeats);
} catch (Exception e) {
LOG.error("Failed to remove taskbeats for " + topologyId, e);
}
} else {
System.out.println(" node of " + rootDir + taskBeats + " isn't existed ");
}
if (zkClusterState.node_existed(tasks, false)) {
try {
zkClusterState.delete_node(tasks);
} catch (Exception e) {
LOG.error("Failed to remove tasks for " + topologyId, e);
}
} else {
System.out.println(" node of " + rootDir + tasks + " isn't existed ");
}
if (zkClusterState.node_existed(taskErrors, false)) {
try {
zkClusterState.delete_node(taskErrors);
} catch (Exception e) {
LOG.error("Failed to remove taskerrors for " + topologyId, e);
}
} else {
System.out.println(" node of " + rootDir + taskErrors + " isn't existed ");
}
if (zkClusterState.node_existed(monitor, false)) {
try {
zkClusterState.delete_node(monitor);
} catch (Exception e) {
LOG.error("Failed to remove monitor for " + topologyId, e);
}
} else {
System.out.println(" node of " + rootDir + monitor + " isn't existed ");
}
} catch (Exception e) {
if (zkClusterState == null) {
LOG.error("Failed to connect ZK ", e);
} else {
LOG.error("Failed to clean topolodyId: " + topologyId + "\n", e);
}
} finally {
if (zkClusterState != null) {
zkClusterState.close();
}
}
}
public static void readData(String path) {
DistributedClusterState zkClusterState = null;
try {
conf.put(Config.STORM_ZOOKEEPER_ROOT, "/");
zkClusterState = new DistributedClusterState(conf);
String data = getData(zkClusterState, path);
if (data == null) {
System.out.println("No data of " + path);
}
StringBuilder sb = new StringBuilder();
sb.append("!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!\n\n");
sb.append("Zk node " + path + "\n");
sb.append("Readable data:" + data + "\n");
sb.append("!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!\n\n");
System.out.println(sb.toString());
} catch (Exception e) {
if (zkClusterState == null) {
LOG.error("Failed to connect ZK ", e);
} else {
LOG.error("Failed to read data " + path + "\n", e);
}
} finally {
if (zkClusterState != null) {
zkClusterState.close();
}
}
}
public static void rmBakTopology(String topologyName) {
DistributedClusterState zkClusterState = null;
try {
zkClusterState = new DistributedClusterState(conf);
String path = Cluster.ASSIGNMENTS_BAK_SUBTREE;
List<String> bakTopologies = zkClusterState.get_children(path, false);
for (String tid : bakTopologies) {
if (tid.equals(topologyName)) {
System.out.println("Find backup " + topologyName);
String topologyPath = assignment_bak_path(topologyName);
zkClusterState.delete_node(topologyPath);
System.out.println("Successfully delete topology " + topologyName + " backup Assignment");
return;
}
}
System.out.println("No backup topology " + topologyName + " Assignment");
} catch (Exception e) {
if (zkClusterState == null) {
LOG.error("Failed to connect ZK ", e);
} else {
LOG.error("Failed to delete old topology " + topologyName, e);
}
} finally {
if (zkClusterState != null) {
zkClusterState.close();
}
}
}
public static String assignment_bak_path(String id) {
return Cluster.ASSIGNMENTS_BAK_SUBTREE + Cluster.ZK_SEPERATOR + id;
}
@SuppressWarnings("rawtypes")
public static ClusterState mk_distributed_cluster_state(Map conf) throws Exception {
return new DistributedClusterState(conf);
}
public static Map<String, String> get_followers(ClusterState cluster_state)
throws Exception {
Map<String, String> ret = Maps.newHashMap();
List<String> followers =
cluster_state.get_children(Cluster.NIMBUS_SLAVE_SUBTREE,
false);
if (followers == null || followers.size() == 0) {
return ret;
}
for (String follower : followers) {
if (follower != null) {
String uptime = new String(cluster_state.get_data(
Cluster.NIMBUS_SLAVE_SUBTREE + Cluster.ZK_SEPERATOR + follower, false));
ret.put(follower, uptime);
}
}
return ret;
}
private static Map conf;
public static void main(String[] args) throws Exception {
if (args.length < 2) {
System.out.println("Invalid parameter");
usage();
return;
}
conf = Utils.readStormConfig();
if (args[0].equalsIgnoreCase(READ_CMD)) {
readData(args[1]);
} else if (args[0].equalsIgnoreCase(RM_CMD)) {
rmBakTopology(args[1]);
} else if (args[0].equalsIgnoreCase(LIST_CMD)) {
list(args[1]);
} else if (args[0].equalsIgnoreCase(CLEAN_CMD)) {
cleanTopology(args[1]);
}
}
}