diff --git a/README.md b/README.md index 51d3e6b56..86abdb544 100644 Binary files a/README.md and b/README.md differ diff --git a/example/sequence-split-merge/conf/conf.prop b/example/sequence-split-merge/conf/conf.prop index c4332be16..f3e005515 100644 --- a/example/sequence-split-merge/conf/conf.prop +++ b/example/sequence-split-merge/conf/conf.prop @@ -8,10 +8,13 @@ topology.debug=false spout.parallel=1 -bolt.parallel=2 +bolt.parallel=1 #send.sleep.second=100 check.sequence=true kryo.enable=false fall.back.on.java.serialization=true enable.split=false + +storm.cluster.mode=local +#topology.enable.classloader=true diff --git a/example/sequence-split-merge/pom.xml b/example/sequence-split-merge/pom.xml index 660035c43..71f515421 100644 --- a/example/sequence-split-merge/pom.xml +++ b/example/sequence-split-merge/pom.xml @@ -11,7 +11,7 @@ UTF-8 - 0.9.6.1 + 0.9.6.3 storm-0.9.2-incubating @@ -29,15 +29,9 @@ - - + + com.alibaba.jstorm jstorm-client-extension @@ -45,12 +39,18 @@ provided - + com.alibaba.jstorm jstorm-client ${jstorm.version} provided + + + org.slf4j + slf4j-log4j12 + + @@ -58,9 +58,28 @@ jstorm-server ${jstorm.version} provided + + + + + ch.qos.logback + logback-classic + 1.0.13 + + + + org.slf4j + log4j-over-slf4j + 1.7.10 - + + junit + junit + 4.10 + test + + + --> 4.0.0 com.alibaba.jstorm jstorm-client-extension - 0.9.6.2 + 0.9.6.3 jar ${project.artifactId}-${project.version} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/client/ConfigExtension.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/client/ConfigExtension.java index 00524036a..f71a4ced3 100644 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/client/ConfigExtension.java +++ b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/client/ConfigExtension.java @@ -162,6 +162,10 @@ public static boolean isEnableCgroup(Map conf) { public static void setUseOldAssignment(Map conf, boolean useOld) { conf.put(USE_OLD_ASSIGNMENT, Boolean.valueOf(useOld)); } + + public static boolean isUseOldAssignment(Map conf) { + return JStormUtils.parseBoolean(conf.get(USE_OLD_ASSIGNMENT), false); + } /** * The supervisor's hostname @@ -195,6 +199,16 @@ public static boolean isEnableTopologyClassLoader(Map conf) { public static void setEnableTopologyClassLoader(Map conf, boolean enable) { conf.put(TOPOLOGY_ENABLE_CLASSLOADER, Boolean.valueOf(enable)); } + + protected static String CLASSLOADER_DEBUG = "classloader.debug"; + + public static boolean isEnableClassloaderDebug(Map conf) { + return JStormUtils.parseBoolean(conf.get(CLASSLOADER_DEBUG), false); + } + + public static void setEnableClassloaderDebug(Map conf, boolean enable) { + conf.put(CLASSLOADER_DEBUG, enable); + } protected static final String CONTAINER_NIMBUS_HEARTBEAT = "container.nimbus.heartbeat"; @@ -313,6 +327,16 @@ public static void setUserDefineAssignment(Map conf, } conf.put(USE_USERDEFINE_ASSIGNMENT, ret); } + + public static List getUserDefineAssignment(Map conf) { + List ret = new ArrayList(); + if (conf.get(USE_USERDEFINE_ASSIGNMENT) == null) + return ret; + for (String worker : (List) conf.get(USE_USERDEFINE_ASSIGNMENT)) { + ret.add(WorkerAssignment.parseFromObj(Utils.from_json(worker))); + } + return ret; + } protected static final String MEMSIZE_PER_WORKER = "worker.memory.size"; @@ -334,11 +358,23 @@ public static void setMemSizePerWorkerByGB(Map conf, long memSize) { long size = memSize * 1024l; setMemSizePerWorkerByMB(conf, size); } + + public static long getMemSizePerWorker(Map conf) { + long size = JStormUtils.parseLong(conf.get(MEMSIZE_PER_WORKER), + JStormUtils.SIZE_1_G * 2); + return size > 0 ? size : JStormUtils.SIZE_1_G * 2; + } + protected static final String CPU_SLOT_PER_WORKER = "worker.cpu.slot.num"; public static void setCpuSlotNumPerWorker(Map conf, int slotNum) { conf.put(CPU_SLOT_PER_WORKER, slotNum); } + + public static int getCpuSlotPerWorker(Map conf) { + int slot = JStormUtils.parseInt(conf.get(CPU_SLOT_PER_WORKER), 1); + return slot > 0 ? slot : 1; + } protected static String TOPOLOGY_PERFORMANCE_METRICS = "topology.performance.metrics"; @@ -550,7 +586,57 @@ public static boolean getTopologyBufferSizeLimited(Map conf) { return true; } - return JStormUtils.parseBoolean(conf.get(TOPOLOGY_BUFFER_SIZE_LIMITED), false); + return JStormUtils.parseBoolean(conf.get(TOPOLOGY_BUFFER_SIZE_LIMITED), true); } + + protected static String SUPERVISOR_SLOTS_PORTS_BASE = "supervisor.slots.ports.base"; + + public static int getSupervisorSlotsPortsBase(Map conf) { + return JStormUtils.parseInt(conf.get(SUPERVISOR_SLOTS_PORTS_BASE), 6800); + } + + // SUPERVISOR_SLOTS_PORTS_BASE don't provide setting function, it must be set by configuration + + protected static String SUPERVISOR_SLOTS_PORT_CPU_WEIGHT = "supervisor.slots.port.cpu.weight"; + public static double getSupervisorSlotsPortCpuWeight(Map conf) { + Object value = conf.get(SUPERVISOR_SLOTS_PORT_CPU_WEIGHT); + Double ret = JStormUtils.convertToDouble(value); + if (ret == null) { + return 1.0; + }else { + return ret; + } + } + // SUPERVISOR_SLOTS_PORT_CPU_WEIGHT don't provide setting function, it must be set by configuration + + protected static String USER_DEFINED_LOG4J_CONF = "user.defined.log4j.conf"; + + public static String getUserDefinedLog4jConf(Map conf) { + return (String)conf.get(USER_DEFINED_LOG4J_CONF); + } + + public static void setUserDefinedLog4jConf(Map conf, String fileName) { + conf.put(USER_DEFINED_LOG4J_CONF, fileName); + } + + protected static String USER_DEFINED_LOGBACK_CONF = "user.defined.logback.conf"; + + public static String getUserDefinedLogbackConf(Map conf) { + return (String)conf.get(USER_DEFINED_LOGBACK_CONF); + } + + public static void setUserDefinedLogbackConf(Map conf, String fileName) { + conf.put(USER_DEFINED_LOGBACK_CONF, fileName); + } + + protected static String TASK_ERROR_INFO_REPORT_INTERVAL = "topology.task.error.report.interval"; + + public static Integer getTaskErrorReportInterval(Map conf) { + return JStormUtils.parseInt(conf.get(TASK_ERROR_INFO_REPORT_INTERVAL), 60); + } + + public static void setTaskErrorReportInterval(Map conf, Integer interval) { + conf.put(TASK_ERROR_INFO_REPORT_INTERVAL, interval); + } } diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/client/WorkerAssignment.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/client/WorkerAssignment.java index eb46e6262..9eac3269e 100644 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/client/WorkerAssignment.java +++ b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/client/WorkerAssignment.java @@ -92,7 +92,7 @@ public void setCpu(int cpu) { @Override public String toJSONString() { - StringBuilder sb = new StringBuilder(); +// StringBuilder sb = new StringBuilder(); // sb.append("["); // sb.append("\"" + this.getNodeId() + "\""); diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/cluster/DistributedClusterState.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/cluster/DistributedClusterState.java index ef8cb4eb7..95224f0dd 100644 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/cluster/DistributedClusterState.java +++ b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/cluster/DistributedClusterState.java @@ -18,6 +18,7 @@ import com.alibaba.jstorm.callback.ClusterStateCallback; import com.alibaba.jstorm.callback.WatcherCallBack; +import com.alibaba.jstorm.utils.JStormUtils; import com.alibaba.jstorm.utils.PathUtils; import com.alibaba.jstorm.zk.Zookeeper; @@ -127,6 +128,8 @@ public void mkdirs(String path) throws Exception { @Override public void set_data(String path, byte[] data) throws Exception { + if (data.length > (JStormUtils.SIZE_1_K * 800)) + throw new Exception("Writing 800k+ data into ZK is not allowed!"); if (zkobj.exists(zk, path, false)) { zkobj.setData(zk, path, data); } else { diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/UserDefMetricData.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/UserDefMetricData.java index cab04e0d7..7ca0860cf 100644 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/UserDefMetricData.java +++ b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/UserDefMetricData.java @@ -5,22 +5,24 @@ import java.util.Map.Entry; import java.io.Serializable; -import com.codahale.metrics.Metric; +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; +import org.apache.log4j.Logger; + import com.codahale.metrics.Gauge; -import com.codahale.metrics.Sampling; -import com.codahale.metrics.Snapshot; import com.codahale.metrics.Timer; import com.codahale.metrics.Counter; import com.codahale.metrics.Histogram; import com.codahale.metrics.Meter; -import com.alibaba.jstorm.client.metric.MetricCallback; import com.alibaba.jstorm.metric.metrdata.*; +import com.alibaba.jstorm.utils.JStormUtils; /** * /storm-zk-root/Monitor/{topologyid}/user/{workerid} data */ public class UserDefMetricData implements Serializable { + private static final Logger LOG = Logger.getLogger(UserDefMetricData.class); private static final long serialVersionUID = 954727168057659270L; @@ -55,9 +57,13 @@ public Map getHistogramDataMap() { public void updateFromGauge(Map> gaugeMap) { for(Entry> entry : gaugeMap.entrySet()) { - GaugeData gaugeData = new GaugeData(); - gaugeData.setValue((Double)(entry.getValue().getValue())); - gaugeDataMap.put(entry.getKey(), gaugeData); + try { + GaugeData gaugeData = new GaugeData(); + gaugeData.setValue(JStormUtils.parseDouble(entry.getValue().getValue())); + gaugeDataMap.put(entry.getKey(), gaugeData); + } catch (Throwable e) { + LOG.error("updateFromGauge exception ", e); + } } } @@ -123,4 +129,10 @@ public void updateFromTimerData(Map timerMap) { timerDataMap.put(entry.getKey(), timerData); } } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } } \ No newline at end of file diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/HttpserverUtils.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/HttpserverUtils.java index 76c41a033..ee7376dcb 100644 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/HttpserverUtils.java +++ b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/HttpserverUtils.java @@ -12,6 +12,8 @@ public class HttpserverUtils { public static final String HTTPSERVER_LOGVIEW_PARAM_CMD_JSTACK = "jstack"; + public static final String HTTPSERVER_LOGVIEW_PARAM_CMD_SHOW_CONF = "showConf"; + public static final String HTTPSERVER_LOGVIEW_PARAM_LOGFILE = "log"; public static final String HTTPSERVER_LOGVIEW_PARAM_POS = "pos"; diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/JStormUtils.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/JStormUtils.java index 390b2603e..17b28cff3 100644 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/JStormUtils.java +++ b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/JStormUtils.java @@ -34,10 +34,12 @@ import org.apache.log4j.FileAppender; import org.apache.log4j.Logger; +import backtype.storm.Config; import backtype.storm.utils.Utils; import com.alibaba.jstorm.callback.AsyncLoopDefaultKill; import com.alibaba.jstorm.callback.RunnableCallback; +import com.alibaba.jstorm.client.ConfigExtension; /** * JStorm utility @@ -58,6 +60,8 @@ public class JStormUtils { public static final int MIN_30 = MIN_1 * 30; public static final int HOUR_1 = MIN_30 * 2; public static final int DAY_1 = HOUR_1 * 24; + + public static final String osName = System.getProperty("os.name"); public static String getErrorInfo(String baseInfo, Exception e) { try { @@ -140,15 +144,35 @@ public static int byteToInt2(byte[] b) { return iOutcome; } + + /** + * LocalMode variable isn't clean, it make the JStormUtils ugly + */ + public static boolean localMode = false; + + public static boolean isLocalMode() { + return localMode; + } + + public static void setLocalMode(boolean localMode) { + JStormUtils.localMode = localMode; + } + + public static void haltProcess(int val) { + Runtime.getRuntime().halt(val); + } public static void halt_process(int val, String msg) { LOG.info("Halting process: " + msg); try { Thread.sleep(1000); } catch (InterruptedException e) { - LOG.error("halt_process", e); } - Runtime.getRuntime().halt(val); + if (localMode && val == 0) { + //throw new RuntimeException(msg); + }else { + haltProcess(val); + } } /** @@ -218,7 +242,7 @@ public static void extract_dir_from_jar(String jarpath, String dir, try { exec_command(cmd); } catch (Exception e) { - LOG.warn("No " + dir + " from " + jarpath + "by cmd:" + cmd + "!" + LOG.warn("No " + dir + " from " + jarpath + " by cmd:" + cmd + "!\n" + e.getMessage()); } @@ -273,6 +297,28 @@ public static void kill_signal(Integer pid, String signal) { } } + /** + * This function is only for linux + * + * @param pid + * @return + */ + public static boolean isProcDead(String pid) { + if (osName.equalsIgnoreCase("Linux") == false) { + return false; + } + + String path = "/proc/" + pid; + File file = new File(path); + + if (file.exists() == false) { + LOG.info("Process " + pid + " is dead"); + return true; + } + + return false; + } + /** * If it is backend, please set resultHandler, such as DefaultExecuteResultHandler * If it is frontend, ByteArrayOutputStream.toString get the result @@ -371,7 +417,7 @@ public void run() { try { launchProcess(cmdlist, environment); } catch (IOException e) { - LOG.error("Failed to run " + cmdlist + ":" + e.getCause(), e); + LOG.error("Failed to run " + command + ":" + e.getCause(), e); } } }).start(); @@ -552,6 +598,27 @@ public static Long parseLong(Object o) { + o.getClass().getName() + " " + o); } } + + public static Double parseDouble(Object o) { + if (o == null) { + return null; + } + + if (o instanceof String) { + return Double.valueOf(String.valueOf(o)); + } else if (o instanceof Integer) { + Number value = (Integer) o; + return value.doubleValue(); + } else if (o instanceof Long) { + Number value = (Long) o; + return value.doubleValue(); + } else if (o instanceof Double) { + return (Double) o; + } else { + throw new RuntimeException("Invalid value " + + o.getClass().getName() + " " + o); + } + } public static Long parseLong(Object o, long defaultValue) { @@ -950,4 +1017,60 @@ public static HashMap filter_val(RunnableCallback fn, } return rtn; } + + public static List getSupervisorPortList(Map conf) { + List portList = (List) conf + .get(Config.SUPERVISOR_SLOTS_PORTS); + if (portList != null && portList.size() > 0) { + return portList; + } + + LOG.info("Generate port list through CPU cores and system memory size"); + + double cpuWeight = ConfigExtension.getSupervisorSlotsPortCpuWeight(conf); + int sysCpuNum = 4; + try { + sysCpuNum = Runtime.getRuntime().availableProcessors(); + }catch(Exception e) { + LOG.info("Failed to get CPU cores, set cpu cores as 4"); + sysCpuNum = 4; + } + int cpuPortNum = (int)(sysCpuNum/cpuWeight); + if (cpuPortNum < 1) { + + LOG.info("Invalid supervisor.slots.port.cpu.weight setting :" + + cpuWeight + ", cpu cores:" + sysCpuNum); + cpuPortNum = 1; + } + + int memPortNum = Integer.MAX_VALUE; + Long physicalMemSize = JStormUtils.getPhysicMemorySize(); + if (physicalMemSize == null) { + LOG.info("Failed to get memory size"); + }else { + LOG.info("Get system memory size :" + physicalMemSize); + long workerMemSize = ConfigExtension.getMemSizePerWorker(conf); + memPortNum = (int)(physicalMemSize/workerMemSize); + if (memPortNum < 1) { + LOG.info("Invalide worker.memory.size setting:" + workerMemSize ); + memPortNum = 4; + }else if (memPortNum < 4){ + LOG.info("System memory is too small for jstorm"); + memPortNum = 4; + } + } + + int portNum = Math.min(cpuPortNum, memPortNum); + if (portNum < 1) { + portNum = 1; + } + + int portBase = ConfigExtension.getSupervisorSlotsPortsBase(conf); + portList = new ArrayList(); + for(int i = 0; i < portNum; i++) { + portList.add(portBase + i); + } + + return portList; + } } diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/NetWorkUtils.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/NetWorkUtils.java index 51483a97f..1a6c649b8 100644 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/NetWorkUtils.java +++ b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/NetWorkUtils.java @@ -4,7 +4,9 @@ import java.net.InetAddress; import java.net.ServerSocket; import java.net.UnknownHostException; +import java.security.InvalidParameterException; +import org.apache.commons.lang.StringUtils; import org.apache.log4j.Logger; /** @@ -97,4 +99,22 @@ public static String ip2Host(String ip) { return address.getHostName(); } + public static boolean equals(String host1, String host2) { + + + if (StringUtils.equalsIgnoreCase(host1, host2) == true) { + return true; + } + + if (host1 == null || host2 == null) { + return false; + } + + String ip1 = host2Ip(host1); + String ip2 = host2Ip(host2); + + return StringUtils.equalsIgnoreCase(ip1, ip2); + + } + } diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/PathUtils.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/PathUtils.java index 87bffd9fb..26cddf589 100644 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/PathUtils.java +++ b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/PathUtils.java @@ -110,5 +110,21 @@ public static List read_dir_contents(String dir) { } return rtn; } + + public static String getCanonicalPath(String fileName) { + String ret = null; + File file = new File(fileName); + if (file.exists()) { + try { + ret = file.getCanonicalPath(); + } catch (IOException e) { + LOG.error("", e); + } + }else { + LOG.warn(fileName + " doesn't exist "); + } + + return ret; + } } diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/zk/ZkTool.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/zk/ZkTool.java index 130afff6f..2d504d92d 100644 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/zk/ZkTool.java +++ b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/zk/ZkTool.java @@ -34,7 +34,7 @@ public static String getData(DistributedClusterState zkClusterState, return null; } - Object obj = Utils.deserialize(data); + Object obj = Utils.deserialize(data, null); return obj.toString(); } diff --git a/jstorm-client/pom.xml b/jstorm-client/pom.xml index 9eaa8652e..6a0f4658c 100644 --- a/jstorm-client/pom.xml +++ b/jstorm-client/pom.xml @@ -5,18 +5,18 @@ com.alibaba.jstorm jstorm-all - 0.9.6.2 + 0.9.6.3 .. + --> 4.0.0 com.alibaba.jstorm jstorm-client - 0.9.6.2 + 0.9.6.3 jar ${project.artifactId}-${project.version} @@ -180,6 +180,11 @@ plexus-compiler-javac 1.8.1 + + com.google.code.gson + gson + 2.3.1 + + --> 4.0.0 com.alibaba.jstorm jstorm-server - 0.9.6.2 + 0.9.6.3 jar ${project.artifactId}-${project.version} jstorm server modules diff --git a/jstorm-server/src/main/java/backtype/storm/LocalCluster.java b/jstorm-server/src/main/java/backtype/storm/LocalCluster.java index d0a0c93b7..303bc11bc 100644 --- a/jstorm-server/src/main/java/backtype/storm/LocalCluster.java +++ b/jstorm-server/src/main/java/backtype/storm/LocalCluster.java @@ -1,5 +1,6 @@ package backtype.storm; +import java.util.Enumeration; import java.util.Map; import org.apache.log4j.BasicConfigurator; @@ -16,15 +17,37 @@ import backtype.storm.generated.TopologyInfo; import backtype.storm.utils.Utils; +import com.alibaba.jstorm.utils.JStormUtils; + public class LocalCluster implements ILocalCluster { public static Logger LOG = Logger.getLogger(LocalCluster.class); private LocalClusterMap state; + protected void setLogger() { + boolean needReset = true; + Logger rootLogger = Logger.getRootLogger(); + if (rootLogger != null) { + Enumeration appenders = rootLogger.getAllAppenders(); + if (appenders.hasMoreElements() == true) { + needReset = false; + } + } + + if (needReset == true) { + BasicConfigurator.configure(); + rootLogger.setLevel(Level.INFO); + } + + } + public LocalCluster() { - BasicConfigurator.configure(); - Logger.getRootLogger().setLevel(Level.INFO); + setLogger(); + + // fix in zk occur Address family not supported by protocol family: connect + System.setProperty("java.net.preferIPv4Stack", "true"); + this.state = LocalUtils.prepareLocalCluster(); if (this.state == null) throw new RuntimeException("prepareLocalCluster error"); @@ -42,6 +65,8 @@ public void submitTopologyWithOpts(String topologyName, Map conf, // TODO Auto-generated method stub if (!Utils.isValidConf(conf)) throw new RuntimeException("Topology conf is not json-serializable"); + JStormUtils.setLocalMode(true); + try { if (submitOpts == null) { state.getNimbus().submitTopology(topologyName, null, @@ -62,7 +87,10 @@ public void submitTopologyWithOpts(String topologyName, Map conf, public void killTopology(String topologyName) { // TODO Auto-generated method stub try { - state.getNimbus().killTopology(topologyName); + // kill topology quickly + KillOptions killOps = new KillOptions(); + killOps.set_wait_secs(0); + state.getNimbus().killTopologyWithOpts(topologyName, killOps); } catch (Exception e) { // TODO Auto-generated catch block LOG.error("fail to kill Topology " + topologyName, e); @@ -121,6 +149,9 @@ public void rebalance(String name, RebalanceOptions options){ @Override public void shutdown() { // TODO Auto-generated method stub + // in order to avoid kill topology's command competition + // it take 10 seconds to remove topology's node + JStormUtils.sleepMs(10 * 1000); this.state.clean(); } diff --git a/jstorm-server/src/main/java/backtype/storm/LocalClusterMap.java b/jstorm-server/src/main/java/backtype/storm/LocalClusterMap.java index bedc219f0..7c29cdbda 100644 --- a/jstorm-server/src/main/java/backtype/storm/LocalClusterMap.java +++ b/jstorm-server/src/main/java/backtype/storm/LocalClusterMap.java @@ -102,7 +102,7 @@ public void clean() { PathUtils.rmr(dir); } catch (IOException e) { // TODO Auto-generated catch block - LOG.error("fail to delete " + dir, e); + LOG.error("Fail to delete " + dir); } } } diff --git a/jstorm-server/src/main/java/backtype/storm/LocalUtils.java b/jstorm-server/src/main/java/backtype/storm/LocalUtils.java index f84aae79a..30b6daa12 100644 --- a/jstorm-server/src/main/java/backtype/storm/LocalUtils.java +++ b/jstorm-server/src/main/java/backtype/storm/LocalUtils.java @@ -13,6 +13,7 @@ import backtype.storm.messaging.IContext; import backtype.storm.utils.Utils; +import com.alibaba.jstorm.client.ConfigExtension; import com.alibaba.jstorm.daemon.nimbus.DefaultInimbus; import com.alibaba.jstorm.daemon.nimbus.NimbusServer; import com.alibaba.jstorm.daemon.supervisor.Supervisor; @@ -93,6 +94,8 @@ private static Map getLocalConf(int port) { conf.put(Config.ZMQ_LINGER_MILLIS, 0); conf.put(Config.TOPOLOGY_ENABLE_MESSAGE_TIMEOUTS, false); conf.put(Config.TOPOLOGY_TRIDENT_BATCH_EMIT_INTERVAL_MILLIS, 50); + ConfigExtension.setSpoutDelayRunSeconds(conf, 0); + ConfigExtension.setTaskCleanupTimeoutSec(conf, 0); return conf; } diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/DelayStatusTransitionCallback.java b/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/DelayStatusTransitionCallback.java index c2c49e898..ed82071e4 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/DelayStatusTransitionCallback.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/DelayStatusTransitionCallback.java @@ -73,7 +73,7 @@ public int getDelaySeconds(Object[] args) { delaySecs = JStormUtils.parseInt(args[0]); } - if (delaySecs == null || delaySecs <= 0) { + if (delaySecs == null || delaySecs < 0) { delaySecs = DelayStatusTransitionCallback.DEFAULT_DELAY_SECONDS; } diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/Cluster.java b/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/Cluster.java index 753336261..53e2cf815 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/Cluster.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/Cluster.java @@ -143,7 +143,7 @@ public static Object maybe_deserialize(byte[] data) { if (data == null) { return null; } - return Utils.deserialize(data); + return Utils.deserialize(data, null); } @SuppressWarnings("rawtypes") diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/Common.java b/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/Common.java index eedd2bc10..3ab090399 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/Common.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/Common.java @@ -11,6 +11,7 @@ import java.util.Set; import backtype.storm.Config; +import backtype.storm.Constants; import backtype.storm.generated.Bolt; import backtype.storm.generated.ComponentCommon; import backtype.storm.generated.ComponentObject; @@ -23,6 +24,7 @@ import backtype.storm.generated.StateSpoutSpec; import backtype.storm.generated.StormTopology; import backtype.storm.generated.StreamInfo; +import backtype.storm.metric.SystemBolt; import backtype.storm.spout.ShellSpout; import backtype.storm.task.IBolt; import backtype.storm.task.ShellBolt; @@ -304,7 +306,11 @@ public static Map acker_inputs( * @param num_tasks * @param ret */ - public static void add_acker(Integer ackerNum, StormTopology ret) { + public static void add_acker(Map stormConf, StormTopology ret) { + String key = Config.TOPOLOGY_ACKER_EXECUTORS; + + Integer ackerNum = JStormUtils.parseInt(stormConf.get(key), 0); + // generate outputs HashMap outputs = new HashMap(); ArrayList fields = new ArrayList(); @@ -319,8 +325,7 @@ public static void add_acker(Integer ackerNum, StormTopology ret) { Map inputs = acker_inputs(ret); // generate acker which will be stored in topology - Bolt acker_bolt = Thrift.mkAckerBolt(inputs, ackerbolt, outputs, - ackerNum); + Bolt acker_bolt = Thrift.mkBolt(inputs, ackerbolt, outputs, ackerNum); // add every bolt two output stream // ACKER_ACK_STREAM_ID/ACKER_FAIL_STREAM_ID @@ -413,19 +418,75 @@ public static void add_system_streams(StormTopology topology) { } } + public static StormTopology add_system_components(StormTopology topology) { + // generate inputs + Map inputs = new HashMap(); + + // generate outputs + HashMap outputs = new HashMap(); + ArrayList fields = new ArrayList(); + + outputs.put(Constants.SYSTEM_TICK_STREAM_ID, + Thrift.outputFields(JStormUtils.mk_list("rate_secs"))); + outputs.put(Constants.METRICS_TICK_STREAM_ID, + Thrift.outputFields(JStormUtils.mk_list("interval"))); + outputs.put(Constants.CREDENTIALS_CHANGED_STREAM_ID, + Thrift.outputFields(JStormUtils.mk_list("creds"))); + + ComponentCommon common = new ComponentCommon(inputs, outputs); + + IBolt ackerbolt = new SystemBolt(); + + Bolt bolt = Thrift.mkBolt(inputs, ackerbolt, outputs, + Integer.valueOf(0)); + + topology.put_to_bolts(Constants.SYSTEM_COMPONENT_ID, bolt); + + add_system_streams(topology); + + return topology; + + } + + public static StormTopology add_metrics_component(StormTopology topology) { + + /** + * @@@ TODO Add metrics consumer bolt + */ + // (defn metrics-consumer-bolt-specs [storm-conf topology] + // (let [component-ids-that-emit-metrics (cons SYSTEM-COMPONENT-ID (keys (all-components topology))) + // inputs (->> (for [comp-id component-ids-that-emit-metrics] + // {[comp-id METRICS-STREAM-ID] :shuffle}) + // (into {})) + // + // mk-bolt-spec (fn [class arg p] + // (thrift/mk-bolt-spec* + // inputs + // (backtype.storm.metric.MetricsConsumerBolt. class arg) + // {} :p p :conf {TOPOLOGY-TASKS p}))] + // + // (map + // (fn [component-id register] + // [component-id (mk-bolt-spec (get register "class") + // (get register "argument") + // (or (get register "parallelism.hint") 1))]) + // + // (metrics-consumer-register-ids storm-conf) + // (get storm-conf TOPOLOGY-METRICS-CONSUMER-REGISTER)))) + return topology; + } + @SuppressWarnings("rawtypes") public static StormTopology system_topology(Map storm_conf, StormTopology topology) throws InvalidTopologyException { StormTopology ret = topology.deepCopy(); - String key = Config.TOPOLOGY_ACKER_EXECUTORS; - - Integer ackercount = JStormUtils.parseInt(storm_conf.get(key), 0); + add_acker(storm_conf, ret); - add_acker(ackercount, ret); + add_metrics_component(ret); - add_system_streams(ret); + add_system_components(ret); return ret; } diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/StormClusterState.java b/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/StormClusterState.java index f0871fe15..8bdcdc218 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/StormClusterState.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/StormClusterState.java @@ -3,12 +3,9 @@ import java.util.List; import java.util.Map; -import backtype.storm.utils.Utils; - import com.alibaba.jstorm.callback.RunnableCallback; import com.alibaba.jstorm.daemon.supervisor.SupervisorInfo; import com.alibaba.jstorm.daemon.worker.WorkerMetricInfo; -import com.alibaba.jstorm.metric.UserDefMetric; import com.alibaba.jstorm.metric.UserDefMetricData; import com.alibaba.jstorm.task.Assignment; import com.alibaba.jstorm.task.AssignmentBak; @@ -48,6 +45,8 @@ public void update_storm(String topology_id, StormStatus new_elems) public void remove_storm_base(String topology_id) throws Exception; public void remove_storm(String topology_id) throws Exception; + + public void try_remove_storm(String topology_id); public List task_ids(String topology_id) throws Exception; @@ -149,7 +148,13 @@ public List task_errors(String topology_id, int task_id) public List monitor_user_workers(String topologyId) throws Exception; + public List monitors() throws Exception; + public TaskMetricInfo get_task_metric(String topologyId, int taskId) throws Exception; public WorkerMetricInfo get_worker_metric(String topologyId, String workerId) throws Exception; + + public List task_error_time(String topologyId, int taskId) throws Exception; + + public String task_error_info(String topologyId, int taskId, long timeStamp) throws Exception; } diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/StormConfig.java b/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/StormConfig.java index 306cbaf85..094f68e9a 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/StormConfig.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/StormConfig.java @@ -2,7 +2,7 @@ import java.io.File; import java.io.IOException; -import java.lang.reflect.*; +import java.lang.reflect.Field; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -11,14 +11,13 @@ import org.apache.commons.io.FileUtils; import org.apache.log4j.Logger; -import com.alibaba.jstorm.utils.EventSampler; -import com.alibaba.jstorm.utils.PathUtils; - import backtype.storm.Config; import backtype.storm.generated.StormTopology; import backtype.storm.utils.LocalState; import backtype.storm.utils.Utils; +import com.alibaba.jstorm.utils.PathUtils; + public class StormConfig { private final static Logger LOG = Logger.getLogger(StormConfig.class); public final static String RESOURCES_SUBDIR = "resources"; diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/StormZkClusterState.java b/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/StormZkClusterState.java index 28bd495c1..3fec09546 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/StormZkClusterState.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/StormZkClusterState.java @@ -207,6 +207,11 @@ public List active_storms() throws Exception { public List monitor_user_workers(String topologyId) throws Exception { return cluster_state.get_children(Cluster.monitor_userdir_path(topologyId), false); } + + @Override + public List monitors() throws Exception { + return cluster_state.get_children(Cluster.MONITOR_SUBTREE, false); + } @Override public List heartbeat_storms() throws Exception { @@ -229,6 +234,30 @@ public void remove_storm(String topologyId) throws Exception { cluster_state.delete_node(Cluster.monitor_path(topologyId)); this.remove_storm_base(topologyId); } + + @Override + public void try_remove_storm(String topologyId) { + teardown_heartbeats(topologyId); + teardown_task_errors(topologyId); + + try { + cluster_state.delete_node(Cluster.assignment_path(topologyId)); + }catch(Exception e) { + LOG.warn("Failed to delete zk Assignment " + topologyId); + } + + try { + cluster_state.delete_node(Cluster.storm_task_root(topologyId)); + }catch(Exception e) { + LOG.warn("Failed to delete zk taskInfo " + topologyId); + } + + try { + cluster_state.delete_node(Cluster.monitor_path(topologyId)); + }catch(Exception e) { + LOG.warn("Failed to delete zk monitor " + topologyId); + } + } @Override public void remove_storm_base(String topologyId) throws Exception { @@ -406,6 +435,21 @@ public void remove_lastErr_time(String topologyId) throws Exception { public List task_error_storms() throws Exception { return cluster_state.get_children(Cluster.TASKERRORS_SUBTREE, false); } + + @Override + public List task_error_time(String topologyId, int taskId) throws Exception { + String path = Cluster.taskerror_path(topologyId, taskId); + cluster_state.mkdirs(path); + return cluster_state.get_children(path, false); + } + + @Override + public String task_error_info(String topologyId, int taskId, long timeStamp) throws Exception { + String path = Cluster.taskerror_path(topologyId, taskId); + cluster_state.mkdirs(path); + path = path + "/" + timeStamp; + return new String(cluster_state.get_data(path, false)); + } @Override public List task_errors(String topologyId, int taskId) diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusData.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusData.java index d1b84b5d3..864519bed 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusData.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusData.java @@ -5,6 +5,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; @@ -62,6 +63,8 @@ public class NimbusData { private final boolean localMode; private volatile boolean isLeader; + + private AtomicBoolean isShutdown = new AtomicBoolean(false); @SuppressWarnings({ "unchecked", "rawtypes" }) public NimbusData(Map conf, TimeCacheMap downloaders, @@ -187,7 +190,13 @@ public void cleanup() { // TODO Auto-generated catch block } - scheduExec.shutdown(); + try { + scheduExec.shutdown(); + }catch(Exception e) { + } + + uploaders.cleanup(); + downloaders.cleanup(); } public INimbus getInimubs() { @@ -218,4 +227,8 @@ public void setLeader(boolean isLeader) { this.isLeader = isLeader; } + public AtomicBoolean getIsShutdown() { + return isShutdown; + } + } diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusServer.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusServer.java index 70ccd2755..6cfe2327d 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusServer.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusServer.java @@ -5,7 +5,6 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -29,15 +28,14 @@ import com.alibaba.jstorm.cluster.StormConfig; import com.alibaba.jstorm.daemon.supervisor.Httpserver; import com.alibaba.jstorm.daemon.worker.hearbeat.SyncContainerHb; -import com.alibaba.jstorm.daemon.worker.metrics.UploadMetricFromZK; -import com.alibaba.jstorm.daemon.worker.metrics.MetricSendClient; import com.alibaba.jstorm.daemon.worker.metrics.AlimonitorClient; +import com.alibaba.jstorm.daemon.worker.metrics.MetricSendClient; +import com.alibaba.jstorm.daemon.worker.metrics.UploadMetricFromZK; import com.alibaba.jstorm.schedule.CleanRunnable; import com.alibaba.jstorm.schedule.FollowerRunnable; import com.alibaba.jstorm.schedule.MonitorRunnable; import com.alibaba.jstorm.utils.JStormServerUtils; import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.SmartThread; /** * @@ -73,9 +71,7 @@ public class NimbusServer { private UploadMetricFromZK uploadMetric; - private List smartThreads = new ArrayList(); - - private AtomicBoolean isShutdown = new AtomicBoolean(false); + private List smartThreads = new ArrayList(); public static void main(String[] args) throws Exception { // read configuration files @@ -346,22 +342,24 @@ private void initUploadMetricThread(NimbusData data) { } public void cleanup() { - if (isShutdown.compareAndSet(false, true) == false) { + if (data.getIsShutdown().getAndSet(true) == true) { LOG.info("Notify to quit nimbus"); return; } LOG.info("Begin to shutdown nimbus"); - for (SmartThread t : smartThreads) { + for (AsyncLoopThread t : smartThreads) { + t.cleanup(); JStormUtils.sleepMs(10); t.interrupt(); - try { - t.join(); - } catch (InterruptedException e) { - LOG.error("join thread", e); - } +// try { +// t.join(); +// } catch (InterruptedException e) { +// LOG.error("join thread", e); +// } + LOG.info("Successfully cleanup " + t.getThread().getName()); } if (serviceHandler != null) { diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusUtils.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusUtils.java index cdf75dcf7..a4a8f878c 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusUtils.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusUtils.java @@ -1,9 +1,10 @@ package com.alibaba.jstorm.daemon.nimbus; -import java.io.File; -import java.io.FileReader; import java.io.BufferedReader; import java.io.Closeable; +import java.io.File; +import java.io.FileReader; +import java.security.InvalidParameterException; import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; @@ -41,8 +42,8 @@ import com.alibaba.jstorm.cluster.StormMonitor; import com.alibaba.jstorm.daemon.supervisor.SupervisorInfo; import com.alibaba.jstorm.daemon.worker.WorkerMetricInfo; -import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; import com.alibaba.jstorm.metric.MetricDef; +import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; import com.alibaba.jstorm.task.Assignment; import com.alibaba.jstorm.task.TaskMetricInfo; import com.alibaba.jstorm.task.TkHbCacheTime; @@ -219,6 +220,7 @@ public static Integer componentParalism(Map stormConf, } } + /** * finalize component's task paralism @@ -229,10 +231,21 @@ public static Integer componentParalism(Map stormConf, * @return */ public static StormTopology normalizeTopology(Map stormConf, - StormTopology topology, boolean fromConf) { + StormTopology topology, boolean fromConf){ StormTopology ret = topology.deepCopy(); + + Map rawComponents = ThriftTopologyUtils.getComponents(topology); Map components = ThriftTopologyUtils.getComponents(ret); + + if (rawComponents.keySet().equals(components.keySet()) == false) { + String errMsg = "Failed to normalize topology binary, maybe due to wrong dependency"; + LOG.info(errMsg + " raw components:" + rawComponents.keySet() + + ", normalized " + components.keySet()); + + throw new InvalidParameterException(errMsg); + } + for (Entry entry : components.entrySet()) { Object component = entry.getValue(); String componentName = entry.getKey(); @@ -242,21 +255,30 @@ public static StormTopology normalizeTopology(Map stormConf, common = ((Bolt) component).get_common(); if (fromConf) { Integer paraNum = ConfigExtension.getBoltParallelism(stormConf, componentName); - if (paraNum != null) common.set_parallelism_hint(paraNum); + if (paraNum != null) { + LOG.info("Set " + componentName + " as " + paraNum); + common.set_parallelism_hint(paraNum); + } } } if (component instanceof SpoutSpec) { common = ((SpoutSpec) component).get_common(); if (fromConf) { Integer paraNum = ConfigExtension.getSpoutParallelism(stormConf, componentName); - if (paraNum != null) common.set_parallelism_hint(paraNum); + if (paraNum != null) { + LOG.info("Set " + componentName + " as " + paraNum); + common.set_parallelism_hint(paraNum); + } } } if (component instanceof StateSpoutSpec) { common = ((StateSpoutSpec) component).get_common(); if (fromConf) { Integer paraNum = ConfigExtension.getSpoutParallelism(stormConf, componentName); - if (paraNum != null) common.set_parallelism_hint(paraNum); + if (paraNum != null) { + LOG.info("Set " + componentName + " as " + paraNum); + common.set_parallelism_hint(paraNum); + } } } @@ -325,33 +347,6 @@ public static void cleanupCorruptTopologies(NimbusData data) } - public static List listStormdistFiles(NimbusData data) - throws Exception { - // get /local-storm-dir/nimbus/stormdist path - String master_stormdist_root = StormConfig.masterStormdistRoot(data - .getConf()); - - // listdir /local-storm-dir/nimbus/stormdist - List files = PathUtils - .read_dir_contents(master_stormdist_root); - - return files; - } - - public static String findTopoFileInStormdist(NimbusData data, String name) throws Exception { - String ret = null; - - List files = listStormdistFiles(data); - - for (String file : files) { - if (file.indexOf(name) != -1) { - ret = file; - break; - } - } - - return ret; - } public static boolean isTaskDead(NimbusData data, String topologyId, Integer taskId) { diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/ServiceHandler.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/ServiceHandler.java index 1b2b3e9a3..4314e6d9e 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/ServiceHandler.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/ServiceHandler.java @@ -19,6 +19,7 @@ import java.util.UUID; import org.apache.commons.io.FileUtils; +import org.apache.commons.lang.StringUtils; import org.apache.log4j.Logger; import org.apache.thrift7.TException; @@ -30,8 +31,8 @@ import backtype.storm.generated.ComponentCommon; import backtype.storm.generated.InvalidTopologyException; import backtype.storm.generated.KillOptions; -import backtype.storm.generated.Nimbus.Iface; import backtype.storm.generated.MonitorOptions; +import backtype.storm.generated.Nimbus.Iface; import backtype.storm.generated.NotAliveException; import backtype.storm.generated.RebalanceOptions; import backtype.storm.generated.SpoutSpec; @@ -40,20 +41,21 @@ import backtype.storm.generated.SubmitOptions; import backtype.storm.generated.SupervisorSummary; import backtype.storm.generated.SupervisorWorkers; +import backtype.storm.generated.TaskMetricData; import backtype.storm.generated.TaskSummary; import backtype.storm.generated.TopologyAssignException; import backtype.storm.generated.TopologyInfo; import backtype.storm.generated.TopologyInitialStatus; -import backtype.storm.generated.TopologySummary; -import backtype.storm.generated.WorkerSummary; import backtype.storm.generated.TopologyMetricInfo; -import backtype.storm.generated.TaskMetricData; +import backtype.storm.generated.TopologySummary; import backtype.storm.generated.WorkerMetricData; +import backtype.storm.generated.WorkerSummary; +import backtype.storm.generated.UserDefMetric; import backtype.storm.utils.BufferFileInputStream; import backtype.storm.utils.TimeCacheMap; import backtype.storm.utils.Utils; -import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.callback.impl.RemoveTransitionCallback; import com.alibaba.jstorm.cluster.Cluster; import com.alibaba.jstorm.cluster.Common; import com.alibaba.jstorm.cluster.DaemonCommon; @@ -62,15 +64,15 @@ import com.alibaba.jstorm.cluster.StormConfig; import com.alibaba.jstorm.daemon.supervisor.SupervisorInfo; import com.alibaba.jstorm.daemon.worker.WorkerMetricInfo; -import com.alibaba.jstorm.utils.JStromServerConfigExtension; +import com.alibaba.jstorm.daemon.worker.metrics.MetricKVMsg; import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; import com.alibaba.jstorm.task.Assignment; import com.alibaba.jstorm.task.TaskInfo; import com.alibaba.jstorm.task.TaskMetricInfo; import com.alibaba.jstorm.utils.FailedAssignTopologyException; import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.JStromServerConfigExtension; import com.alibaba.jstorm.utils.NetWorkUtils; +import com.alibaba.jstorm.utils.PathUtils; import com.alibaba.jstorm.utils.Thrift; import com.alibaba.jstorm.utils.TimeUtils; @@ -177,13 +179,14 @@ public void submitTopologyWithOpts(String topologyname, stormConf = NimbusUtils.normalizeConf(conf, serializedConf, topology); + LOG.info("Normalized configuration:" + stormConf); Map totalStormConf = new HashMap( conf); totalStormConf.putAll(stormConf); StormTopology normalizedTopology = NimbusUtils.normalizeTopology( - stormConf, topology, false); + stormConf, topology, true); // this validates the structure of the topology Common.validate_basic(normalizedTopology, totalStormConf, @@ -247,112 +250,6 @@ public void submitTopologyWithOpts(String topologyname, } - @ Override - public void submitTopologyAfterRestart(String topologyname, - String jsonConf) - throws InvalidTopologyException,TopologyAssignException, TException { - LOG.info("Restart " + topologyname); - // @@@ Move validate topologyname in client code - try { - checkTopologyActive(data, topologyname, false); - } catch (AlreadyAliveException e) { - LOG.info("Fail to kill " + topologyname + " before restarting"); - return; - } catch (Throwable e) { - LOG.info("Failed to check whether topology is alive or not", e); - throw new TException(e); - } - - int counter = data.getSubmittedCount().incrementAndGet(); - String topologyId = topologyname + "-" + counter + "-" - + TimeUtils.current_time_secs(); - - String stormDistDir = null; - String stormTmpDir = null; - try { - // For topology restart, when conf is changed, only stormconf.ser - // and stormcode.ser will be updated. storm.jar will be kept the same. - - // Copy storm files back to stormdist dir from the tmp dir - stormDistDir = StormConfig.masterStormdistRoot(conf, topologyId); - stormTmpDir = StormConfig.masterStormTmpRoot(conf, topologyname); - FileUtils.copyDirectory(new File(stormTmpDir), new File(stormDistDir)); - StormTopology topology = StormConfig.read_nimbus_topology_code(conf, topologyId); - - if (jsonConf != null) { - // Update stormconf.ser - Map serConf = StormConfig.read_nimbusTmp_topology_conf(conf, topologyname); - Map newSerConf = (Map) JStormUtils - .from_json(jsonConf); - serConf.putAll(newSerConf); - - Map stormConf = NimbusUtils.normalizeConf(conf, serConf, - topology); - - File stormConfFile = new File(StormConfig.stormconf_path(stormDistDir)); - if (stormConfFile.exists()) stormConfFile.delete(); - FileUtils.writeByteArrayToFile(stormConfFile, Utils.serialize(stormConf)); - - // Update stormcode.ser - StormTopology normalizedTopology = NimbusUtils.normalizeTopology( - stormConf, topology, true); - File stormCodeFile = new File(StormConfig.stormcode_path(stormDistDir)); - if (stormCodeFile.exists()) stormCodeFile.delete(); - FileUtils.writeByteArrayToFile(stormCodeFile, Utils.serialize(normalizedTopology)); - } - - // generate TaskInfo for every bolt or spout in ZK - // /ZK/tasks/topoologyId/xxx - StormClusterState stormClusterState = data.getStormClusterState(); - setupZkTaskInfo(conf, topologyId, stormClusterState); - - // make assignments for a topology - LOG.info("Submit for " + topologyname + " with conf " + jsonConf); - makeAssignment(topologyname, topologyId, TopologyInitialStatus.ACTIVE); - } catch (FailedAssignTopologyException e) { - StringBuilder sb = new StringBuilder(); - sb.append("Fail to sumbit topology, Root cause:"); - if (e.getMessage() == null) { - sb.append("submit timeout"); - } else { - sb.append(e.getMessage()); - } - - sb.append("\n\n"); - sb.append("topologyId:" + topologyId + "\n"); - LOG.error(sb.toString(), e); - throw new TopologyAssignException(sb.toString()); - } catch (InvalidParameterException e) { - StringBuilder sb = new StringBuilder(); - sb.append("Fail to sumbit topology "); - sb.append(e.getMessage()); - sb.append(", cause:" + e.getCause()); - sb.append("\n\n"); - sb.append("topologyId:" + topologyId + "\n"); - LOG.error(sb.toString(), e); - throw new InvalidParameterException(sb.toString()); - } catch (Throwable e) { - StringBuilder sb = new StringBuilder(); - sb.append("Fail to sumbit topology "); - sb.append(e.getMessage()); - sb.append(", cause:" + e.getCause()); - sb.append("\n\n"); - sb.append("topologyId:" + topologyId + "\n"); - LOG.error(sb.toString(), e); - throw new TopologyAssignException(sb.toString()); - } finally { - if (stormTmpDir != null) { - try { - File dir = new File(stormTmpDir); - if (dir.exists()) FileUtils.deleteDirectory(dir); - } catch (Exception e) { - LOG.error("Failed to delete stormTmpDir=" + stormTmpDir, e); - } - } - } - - } - /** * kill topology * @@ -479,51 +376,106 @@ public void rebalance(String topologyName, RebalanceOptions options) } @Override - public void restart(String name, String jsonConf) throws NotAliveException, - InvalidTopologyException, TopologyAssignException, org.apache.thrift7.TException { + public void restart(String name, String jsonConf) throws NotAliveException, + InvalidTopologyException, TopologyAssignException, + org.apache.thrift7.TException { + LOG.info("Begin to restart " + name + ", new configuration:" + jsonConf); + + // 1. get topologyId + StormClusterState stormClusterState = data.getStormClusterState(); String topologyId = null; - - // Copy topology files into a tmp dir and copy the new conf.ser to this dir try { - topologyId = NimbusUtils.findTopoFileInStormdist(data, name); - if (topologyId != null) { - String srcDir = StormConfig.masterStormdistRoot(conf, topologyId); - String destDir = StormConfig.masterStormTmpRoot(conf, name); - FileUtils.copyDirectory(new File(srcDir), new File(destDir)); - } else { - String errorInfo = "Topology=" + name + " is not exist!"; - throw new InvalidTopologyException(errorInfo); - } - } catch (Exception e) { - LOG.info("InvalidTopologyException: " + e.getMessage()); - throw new InvalidTopologyException(e.getMessage()); + topologyId = Cluster.get_topology_id(stormClusterState, + name); + } catch (Exception e2) { + topologyId = null; } - + if (topologyId == null) { + LOG.info("No topology of " + name); + throw new NotAliveException("No topology of " + name); + } + // Restart the topology: Deactivate -> Kill -> Submit - // 1. Deactivate + // 2. Deactivate deactivate(name); JStormUtils.sleepMs(5000); - - // 2. Kill - KillOptions options = new KillOptions(); - options.set_wait_secs(1); - killTopologyWithOpts(name, options); - // Wait for supervisors to kill the old topology - int delay = (JStormUtils.parseInt(conf.get( - Config.SUPERVISOR_MONITOR_FREQUENCY_SECS)))*1000 + 5000; - JStormUtils.sleepMs(delay); - - // 3. Submit - submitTopologyAfterRestart(name, jsonConf); + LOG.info("Deactivate " + name); + + // 3. backup old jar/configuration/topology + StormTopology topology = null; + + Map topologyConf = null; + String topologyCodeLocation = null; + try { + topology = StormConfig.read_nimbus_topology_code(conf, topologyId); + + topologyConf = StormConfig.read_nimbus_topology_conf(conf, + topologyId); + if (jsonConf != null) { + Map newConf = (Map) JStormUtils + .from_json(jsonConf); + topologyConf.putAll(newConf); + } + + // Copy storm files back to stormdist dir from the tmp dir + String oldDistDir = StormConfig.masterStormdistRoot(conf, + topologyId); + String parent = StormConfig.masterInbox(conf); + topologyCodeLocation = parent + PathUtils.SEPERATOR + topologyId; + FileUtils.forceMkdir(new File(topologyCodeLocation)); + FileUtils.cleanDirectory(new File(topologyCodeLocation)); + FileUtils.copyDirectory(new File(oldDistDir), new File( + topologyCodeLocation)); + + + LOG.info("Successfully read old jar/conf/topology " + name); + } catch (Exception e) { + LOG.error("Failed to read old jar/conf/topology", e); + if (topologyCodeLocation != null) { + try { + PathUtils.rmr(topologyCodeLocation); + } catch (IOException e1) { + + } + } + throw new TException("Failed to read old jar/conf/topology "); + + } + + // 4. Kill + // directly use remove command to kill, more stable than issue kill cmd + RemoveTransitionCallback killCb = new RemoveTransitionCallback(data, + topologyId); + killCb.execute(new Object[0]); + LOG.info("Successfully kill the topology " + name); + + // 5. submit + try { + submitTopology(name, topologyCodeLocation , + JStormUtils.to_json(topologyConf), topology); + + } catch (AlreadyAliveException e) { + LOG.info("Failed to kill the topology" + name); + throw new TException("Failed to kill the topology" + name); + } finally { + try { + PathUtils.rmr(topologyCodeLocation); + } catch (IOException e1) { + + } + } + } @Override public void beginLibUpload(String libName) throws TException { try { + String parent = PathUtils.parent_path(libName); + PathUtils.local_mkdirs(parent); data.getUploaders().put(libName, Channels.newChannel(new FileOutputStream(libName))); LOG.info("Begin upload file from client to " + libName); - } catch (FileNotFoundException e) { + } catch (Exception e) { // TODO Auto-generated catch block LOG.error("Fail to upload jar " + libName, e); throw new TException(e); @@ -873,6 +825,58 @@ public SupervisorWorkers getSupervisorWorkers(String host) throw new TException(e); } } + + + + /** + * Get TopologyInfo, it contain all data of the topology running status + * + * @return TopologyInfo + */ + public TopologyInfo getTopologyInfo(String topologyId, StormBase base) + throws Exception { + + StormClusterState stormClusterState = data.getStormClusterState(); + + TopologyInfo topologyInfo = new TopologyInfo(); + + topologyInfo.set_id(topologyId); + topologyInfo.set_name(base.getStormName()); + topologyInfo.set_uptime_secs(TimeUtils.time_delta(base + .getLanchTimeSecs())); + topologyInfo.set_status(base.getStatusString()); + + // get topology's Assignment + Assignment assignment = stormClusterState.assignment_info( + topologyId, null); + if (assignment == null) { + throw new TException("Failed to get StormBase from ZK of " + + topologyId); + } + + // get topology's map + Map taskInfo = Cluster.topology_task_info( + stormClusterState, topologyId); + + Map tasks = NimbusUtils.mkTaskSummary( + stormClusterState, assignment, taskInfo, topologyId); + List taskSumms = new ArrayList(); + for (Entry entry : tasks.entrySet()) { + taskSumms.add(entry.getValue()); + } + topologyInfo.set_tasks(taskSumms); + List workers = NimbusUtils.mkWorkerSummary( + topologyId, assignment, tasks); + topologyInfo.set_workers(workers); + + // get user defined metrics data + List udm = new ArrayList(); + udm = getUserDefMetrics(topologyId); + topologyInfo.set_userDefMetric(udm); + + return topologyInfo; + + } /** * Get TopologyInfo, it contain all data of the topology running status @@ -883,8 +887,6 @@ public SupervisorWorkers getSupervisorWorkers(String host) public TopologyInfo getTopologyInfo(String topologyId) throws NotAliveException, TException { - TopologyInfo topologyInfo = new TopologyInfo(); - StormClusterState stormClusterState = data.getStormClusterState(); try { @@ -894,36 +896,7 @@ public TopologyInfo getTopologyInfo(String topologyId) if (base == null) { throw new NotAliveException("No topology of " + topologyId); } - topologyInfo.set_id(topologyId); - topologyInfo.set_name(base.getStormName()); - topologyInfo.set_uptime_secs(TimeUtils.time_delta(base - .getLanchTimeSecs())); - topologyInfo.set_status(base.getStatusString()); - - // get topology's Assignment - Assignment assignment = stormClusterState.assignment_info( - topologyId, null); - if (assignment == null) { - throw new TException("Failed to get StormBase from ZK of " - + topologyId); - } - - // get topology's map - Map taskInfo = Cluster.topology_task_info( - stormClusterState, topologyId); - - Map tasks = NimbusUtils.mkTaskSummary( - stormClusterState, assignment, taskInfo, topologyId); - List taskSumms = new ArrayList(); - for (Entry entry : tasks.entrySet()) { - taskSumms.add(entry.getValue()); - } - topologyInfo.set_tasks(taskSumms); - List workers = NimbusUtils.mkWorkerSummary( - topologyId, assignment, tasks); - topologyInfo.set_workers(workers); - - return topologyInfo; + return getTopologyInfo(topologyId, base); } catch (TException e) { LOG.info("Failed to get topologyInfo " + topologyId, e); throw e; @@ -933,6 +906,38 @@ public TopologyInfo getTopologyInfo(String topologyId) } } + + @Override + public TopologyInfo getTopologyInfoByName(String topologyName) + throws NotAliveException, TException { + + StormClusterState stormClusterState = data.getStormClusterState(); + + try { + // get all active topology's StormBase + Map bases = Cluster + .topology_bases(stormClusterState); + for (Entry entry : bases.entrySet()) { + + String topologyId = entry.getKey(); + StormBase base = entry.getValue(); + + if (StringUtils.equals(topologyName, base.getStormName()) == false) { + continue; + } + + return getTopologyInfo(topologyId, base); + } + } catch (TException e) { + LOG.info("Failed to get topologyInfo " + topologyName, e); + throw e; + } catch (Exception e) { + LOG.info("Failed to get topologyInfo " + topologyName, e); + throw new TException("Failed to get topologyInfo" + topologyName); + } + + throw new NotAliveException("No topology of " + topologyName); + } /** * get topology configuration @@ -1077,6 +1082,25 @@ private void setupStormCode(Map conf, String topologyId, new File(StormConfig.stormconf_path(stormroot)), Utils.serialize(stormConf)); } + + private boolean copyLibJars(String tmpJarLocation, + String stormroot) throws IOException { + String srcLibPath = StormConfig.stormlib_path(tmpJarLocation); + String destLibPath = StormConfig.stormlib_path(stormroot); + LOG.info("Begin to copy from " + srcLibPath + " to " + destLibPath); + + File srcFile = new File(srcLibPath); + if (srcFile.exists() == false) { + LOG.info("No lib jars " + srcLibPath); + return false; + } + File destFile = new File(destLibPath); + FileUtils.copyDirectory(srcFile, destFile); + + PathUtils.rmr(srcLibPath); + LOG.info("Successfully copy libs " + destLibPath); + return true; + } /** * Copy jar to /local-dir/nimbus/topologyId/stormjar.jar @@ -1089,26 +1113,38 @@ private void setupStormCode(Map conf, String topologyId, private void setupJar(Map conf, String tmpJarLocation, String stormroot) throws IOException { if (!StormConfig.local_mode(conf)) { - String[] pathCache = tmpJarLocation.split("/"); - String jarPath = tmpJarLocation + "/stormjar-" - + pathCache[pathCache.length - 1] + ".jar"; + boolean existLibs = copyLibJars(tmpJarLocation, stormroot); + + String jarPath = null; + List files = PathUtils.read_dir_contents(tmpJarLocation); + for (String file : files) { + if (file.endsWith(".jar")) { + jarPath = tmpJarLocation + PathUtils.SEPERATOR + file; + break; + } + } + + if (jarPath == null ) { + if ( existLibs == false) { + throw new IllegalArgumentException("No jar under " + tmpJarLocation); + }else { + LOG.info("No submit jar"); + return ; + } + } + File srcFile = new File(jarPath); if (!srcFile.exists()) { throw new IllegalArgumentException(jarPath + " to copy to " + stormroot + " does not exist!"); } + String path = StormConfig.stormjar_path(stormroot); File destFile = new File(path); FileUtils.copyFile(srcFile, destFile); srcFile.delete(); - String libPath = StormConfig.stormlib_path(stormroot); - srcFile = new File(tmpJarLocation); - if (!srcFile.exists()) { - throw new IllegalArgumentException(tmpJarLocation - + " to copy to " + stormroot + " does not exist!"); - } - destFile = new File(libPath); - FileUtils.copyDirectory(srcFile, destFile); + + return ; } } @@ -1128,7 +1164,7 @@ public void setupZkTaskInfo(Map conf, String topologyId, Map taskToComponetId = mkTaskComponentAssignments( conf, topologyId); - if (taskToComponetId == null) { + if (taskToComponetId == null || taskToComponetId.size() == 0) { throw new InvalidTopologyException("Failed to generate TaskIDs map"); } @@ -1296,5 +1332,22 @@ public TopologyMetricInfo getTopologyMetric(String topologyId) throws NotAliveEx return topologyMetricInfo; } - + + private List getUserDefMetrics(String topologyId) { + List userDefMetrics = null; + + StormClusterState clusterState = data.getStormClusterState(); + + try { + MetricKVMsg topologyMetricMsg = MetricKVMsg.getMetricKVMsg(topologyId, clusterState); + userDefMetrics = topologyMetricMsg.convertToUserDefMetric(); + } catch (Exception e) { + LOG.error("Failed to get user defined metrics for topology=" + topologyId); + } + + return userDefMetrics; + + } + + } diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/StatusTransition.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/StatusTransition.java index 461913882..acd1d82cb 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/StatusTransition.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/StatusTransition.java @@ -46,6 +46,12 @@ public void transition(String topologyid, boolean errorOnNoTransition, lock = new Object(); topologyLocks.put(topologyid, lock); } + + if (data.getIsShutdown().get() == true) { + LOG.info("Nimbus is in shutdown, skip this event " + + topologyid + ":" +changeStatus); + return ; + } synchronized (lock) { transitionLock(topologyid, errorOnNoTransition, changeStatus, args); diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/TopologyAssign.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/TopologyAssign.java index 41bdb6561..67e71307b 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/TopologyAssign.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/TopologyAssign.java @@ -16,7 +16,6 @@ import org.apache.log4j.Logger; import backtype.storm.generated.StormTopology; -import backtype.storm.generated.ThriftResourceType; import backtype.storm.scheduler.WorkerSlot; import com.alibaba.jstorm.client.ConfigExtension; @@ -24,19 +23,17 @@ import com.alibaba.jstorm.cluster.StormBase; import com.alibaba.jstorm.cluster.StormClusterState; import com.alibaba.jstorm.cluster.StormConfig; -import com.alibaba.jstorm.cluster.StormStatus; import com.alibaba.jstorm.cluster.StormMonitor; +import com.alibaba.jstorm.cluster.StormStatus; import com.alibaba.jstorm.daemon.supervisor.SupervisorInfo; import com.alibaba.jstorm.schedule.IToplogyScheduler; import com.alibaba.jstorm.schedule.TopologyAssignContext; import com.alibaba.jstorm.schedule.default_assign.DefaultTopologyScheduler; -import com.alibaba.jstorm.utils.JStromServerConfigExtension; import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; import com.alibaba.jstorm.task.Assignment; import com.alibaba.jstorm.task.AssignmentBak; import com.alibaba.jstorm.utils.FailedAssignTopologyException; import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.JStromServerConfigExtension; import com.alibaba.jstorm.utils.PathUtils; import com.alibaba.jstorm.utils.TimeUtils; @@ -172,12 +169,8 @@ public void cleanupDisappearedTopology() throws Exception { for (String topologyId : cleanupIds) { LOG.info("Cleaning up " + topologyId); - - // remove ZK nodes /taskbeats/topologyId and - // /taskerror/topologyId - clusterState.teardown_heartbeats(topologyId); - clusterState.teardown_task_errors(topologyId); - + + clusterState.try_remove_storm(topologyId); // nimbusData.getTaskHeartbeatsCache().remove(topologyId); @@ -188,7 +181,7 @@ public void cleanupDisappearedTopology() throws Exception { // delete topologyId local dir PathUtils.rmr(master_stormdist_root); } catch (IOException e) { - LOG.error("Failed to delete " + master_stormdist_root + ",", e); + LOG.warn("Failed to delete " + master_stormdist_root + ",", e); } } } @@ -203,9 +196,12 @@ public void cleanupDisappearedTopology() throws Exception { private Set get_cleanup_ids(StormClusterState clusterState, List active_topologys) throws Exception { - // get ZK /taskbeats/topology list and /taskerror/topology list + + List task_ids = clusterState.task_storms(); List heartbeat_ids = clusterState.heartbeat_storms(); List error_ids = clusterState.task_error_storms(); + List assignment_ids = clusterState.assignments(null); + List monitor_ids = clusterState.monitors(); String master_stormdist_root = StormConfig .masterStormdistRoot(nimbusData.getConf()); @@ -216,15 +212,31 @@ private Set get_cleanup_ids(StormClusterState clusterState, // Set assigned_ids = // JStormUtils.listToSet(clusterState.active_storms()); Set to_cleanup_ids = new HashSet(); + + if (task_ids != null) { + to_cleanup_ids.addAll(task_ids); + } + if (heartbeat_ids != null) { to_cleanup_ids.addAll(heartbeat_ids); } + if (error_ids != null) { to_cleanup_ids.addAll(error_ids); } + + if (assignment_ids != null) { + to_cleanup_ids.addAll(assignment_ids); + } + + if (monitor_ids != null) { + to_cleanup_ids.addAll(monitor_ids); + } + if (code_ids != null) { to_cleanup_ids.addAll(code_ids); } + if (active_topologys != null) { to_cleanup_ids.removeAll(active_topologys); } @@ -329,6 +341,7 @@ protected TopologyAssignContext prepareTopologyAssign( aliveTasks = getAliveTasks(topologyId, allTaskIds); unstoppedTasks = getUnstoppedSlots(aliveTasks, supInfos, existingAssignment); + deadTasks.addAll(allTaskIds); deadTasks.removeAll(aliveTasks); } @@ -360,13 +373,11 @@ protected TopologyAssignContext prepareTopologyAssign( unstoppedWorkers = getUnstoppedWorkers(unstoppedTasks, existingAssignment); ret.setUnstoppedWorkers(unstoppedWorkers); - } else { ret.setAssignType(TopologyAssignContext.ASSIGN_TYPE_MONITOR); - unstoppedWorkers = getUnstoppedWorkers(allTaskIds, + unstoppedWorkers = getUnstoppedWorkers(aliveTasks, existingAssignment); ret.setUnstoppedWorkers(unstoppedWorkers); - } } diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/Heartbeat.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/Heartbeat.java index 161484d90..8276fd30b 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/Heartbeat.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/Heartbeat.java @@ -65,11 +65,12 @@ public Heartbeat(Map conf, StormClusterState stormClusterState, .get(Config.SUPERVISOR_HEARTBEAT_FREQUENCY_SECS)); initSupervisorInfo(conf); + + LOG.info("Successfully init supervisor heartbeat thread, " + supervisorInfo); } private void initSupervisorInfo(Map conf) { - List portList = (List) conf - .get(Config.SUPERVISOR_SLOTS_PORTS); + List portList = JStormUtils.getSupervisorPortList(conf); if (!StormConfig.local_mode(conf)) { Set ports = JStormUtils.listToSet(portList); diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/Httpserver.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/Httpserver.java index d8e79ced0..8c60b7336 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/Httpserver.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/Httpserver.java @@ -14,6 +14,7 @@ import java.net.URI; import java.nio.MappedByteBuffer; import java.nio.channels.FileChannel; +import java.util.ArrayList; import java.util.Date; import java.util.HashMap; import java.util.List; @@ -25,7 +26,9 @@ import org.apache.commons.lang.StringUtils; import org.apache.log4j.Logger; +import backtype.storm.Constants; import backtype.storm.daemon.Shutdownable; +import backtype.storm.utils.Utils; import com.alibaba.jstorm.client.ConfigExtension; import com.alibaba.jstorm.daemon.worker.Worker; @@ -33,6 +36,7 @@ import com.alibaba.jstorm.utils.HttpserverUtils; import com.alibaba.jstorm.utils.JStormUtils; import com.alibaba.jstorm.utils.Pair; +import com.alibaba.jstorm.utils.PathUtils; import com.alibaba.jstorm.utils.TimeFormat; import com.google.common.base.Joiner; import com.google.common.collect.Maps; @@ -56,11 +60,39 @@ public Httpserver(int port, Map conf) { static class LogHandler implements HttpHandler { private String logDir; + private String stormHome; + private ArrayList accessDirs = new ArrayList(); Map conf; public LogHandler(Map conf) { logDir = JStormUtils.getLogDir(); + String logDirPath = PathUtils.getCanonicalPath(logDir); + if (logDirPath == null) { + accessDirs.add(logDir); + }else { + accessDirs.add(logDirPath); + } + + stormHome = System.getProperty("jstorm.home"); + if (stormHome != null) { + String stormHomePath = PathUtils.getCanonicalPath(stormHome); + if (stormHomePath == null) { + accessDirs.add(stormHome); + }else { + accessDirs.add(stormHomePath); + } + } + + String confDir = System.getProperty(Constants.JSTORM_CONF_DIR); + if (StringUtils.isBlank(confDir) == false) { + String confDirPath = PathUtils.getCanonicalPath(confDir); + if (confDirPath != null) { + accessDirs.add(confDirPath); + } + } + + this.conf = conf; LOG.info("logview logDir=" + logDir); // +++ @@ -82,6 +114,7 @@ public void handlFailure(HttpExchange t, String errorMsg) public void handle(HttpExchange t) throws IOException { URI uri = t.getRequestURI(); Map paramMap = parseRawQuery(uri.getRawQuery()); + LOG.info("Receive command " + paramMap); String cmd = paramMap .get(HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_CMD); @@ -100,12 +133,34 @@ public void handle(HttpExchange t) throws IOException { }else if (HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_CMD_JSTACK.equals(cmd)) { handleJstack(t, paramMap); return ; + }else if (HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_CMD_SHOW_CONF.equals(cmd)) { + handleShowConf(t, paramMap); + return ; } handlFailure(t, "Bad Request, Not support command type " + cmd); return; } + private void accessCheck(String fileName) throws IOException{ + File file =new File(fileName); + String canonicalPath = file.getCanonicalPath(); + + + boolean isChild = false; + for (String dir : accessDirs) { + if (canonicalPath.indexOf(dir) >= 0) { + isChild = true; + break; + } + } + + if (isChild == false) { + LOG.error("Access one disallowed path: " + canonicalPath); + throw new IOException("Destination file/path is not accessible."); + } + } + private Map parseRawQuery(String uriRawQuery) { Map paramMap = Maps.newHashMap(); @@ -152,6 +207,7 @@ private Pair queryLog(HttpExchange t, } String logFile = Joiner.on(File.separator).join(logDir, fileParam); + accessCheck(logFile); FileChannel fc = null; MappedByteBuffer fout = null; long fileSize = 0; @@ -172,7 +228,7 @@ private Pair queryLog(HttpExchange t, position = pos; } } catch (Exception e) { - LOG.warn("Invalide position "); + LOG.warn("Invalide position " + position); } if (position < 0) { position = 0L; @@ -216,6 +272,7 @@ byte[] getJSonFiles(String dir) throws Exception { if (dir != null) { path = path + File.separator + dir; } + accessCheck(path); LOG.info("List dir " + path); @@ -263,6 +320,7 @@ void handleListDir(HttpExchange t, Map paramMap) String dir = paramMap.get(HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_DIR); filesJson = getJSonFiles(dir); } catch (Exception e) { + LOG.error("Failed to list files", e); handlFailure(t, "Failed to get file list"); return; } @@ -344,7 +402,25 @@ void handleJstack(HttpExchange t, Map paramMap) os.write(data); os.close(); } + + void handleShowConf(HttpExchange t, Map paramMap) + throws IOException { + byte[] json = "Failed to get configuration".getBytes(); + try { + String tmp = Utils.to_json(conf); + json = tmp.getBytes(); + } catch (Exception e) { + LOG.error("Failed to get configuration", e); + handlFailure(t, "Failed to get configuration"); + return; + } + + t.sendResponseHeaders(HttpURLConnection.HTTP_OK, json.length); + OutputStream os = t.getResponseBody(); + os.write(json); + os.close(); + } }// LogHandler public void start() { diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/ShutdownWork.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/ShutdownWork.java index c750b5049..2bd28f05f 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/ShutdownWork.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/ShutdownWork.java @@ -47,11 +47,7 @@ public void shutWorker(Map conf, String supervisorId, LOG.info("Begin to shut down " + topologyId + ":" + workerId); try { - // STORM-LOCAL-DIR/workers/workerId/pids - String workerPidPath = StormConfig.worker_pids_root(conf, - workerId); - - List pids = PathUtils.read_dir_contents(workerPidPath); + List pids = getPid(conf, workerId); workerId2Pids.put(workerId, pids); String threadPid = workerThreadPids.get(workerId); @@ -123,4 +119,22 @@ public void tryCleanupWorkerDir(Map conf, String workerId) { + ". Will retry later"); } } + + /** + * When worker has been started by manually and supervisor, + * it will return multiple pid + * + * @param conf + * @param workerId + * @return + * @throws IOException + */ + public List getPid(Map conf, String workerId) throws IOException { + String workerPidPath = StormConfig.worker_pids_root(conf, + workerId); + + List pids = PathUtils.read_dir_contents(workerPidPath); + + return pids; + } } diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/Supervisor.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/Supervisor.java index 59ae5eb56..fddcc842b 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/Supervisor.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/Supervisor.java @@ -1,14 +1,10 @@ package com.alibaba.jstorm.daemon.supervisor; import java.io.File; -import java.io.IOException; import java.util.Map; import java.util.UUID; import java.util.Vector; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.commons.io.FileUtils; @@ -34,8 +30,6 @@ import com.alibaba.jstorm.event.EventManagerPusher; import com.alibaba.jstorm.utils.JStormServerUtils; import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.NetWorkUtils; -import com.alibaba.jstorm.utils.SmartThread; /** * @@ -104,7 +98,7 @@ public SupervisorManger mkSupervisor(Map conf, IContext sharedContext) localState.put(Common.LS_ID, supervisorId); } - Vector threads = new Vector(); + Vector threads = new Vector(); // Step 5 create HeartBeat // every supervisor.heartbeat.frequency.secs, write SupervisorInfo to ZK @@ -156,23 +150,27 @@ public SupervisorManger mkSupervisor(Map conf, IContext sharedContext) // threads.add(syncProcessThread); - //Step 7 start httpserver - int port = ConfigExtension.getSupervisorDeamonHttpserverPort(conf); - Httpserver httpserver = new Httpserver(port, conf); - httpserver.start(); - - //Step 8 start uploading every 60 secs - MetricSendClient client; - if (ConfigExtension.isAlimonitorMetricsPost(conf)) { - client = new AlimonitorClient(AlimonitorClient.DEFAUT_ADDR, - AlimonitorClient.DEFAULT_PORT, true); - } else { - client = new MetricSendClient(); + Httpserver httpserver = null; + if (StormConfig.local_mode(conf) == false) { + //Step 7 start httpserver + int port = ConfigExtension.getSupervisorDeamonHttpserverPort(conf); + httpserver = new Httpserver(port, conf); + httpserver.start(); + + //Step 8 start uploading every 60 secs + MetricSendClient client; + if (ConfigExtension.isAlimonitorMetricsPost(conf)) { + client = new AlimonitorClient(AlimonitorClient.DEFAUT_ADDR, + AlimonitorClient.DEFAULT_PORT, true); + } else { + client = new MetricSendClient(); + } + UploadSupervMetric uploadMetric = new UploadSupervMetric(conf, stormClusterState, + supervisorId, active, 60, client); + AsyncLoopThread uploadMetricThread = new AsyncLoopThread(uploadMetric); + threads.add(uploadMetricThread); } - UploadSupervMetric uploadMetric = new UploadSupervMetric(conf, stormClusterState, - supervisorId, active, 60, client); - AsyncLoopThread uploadMetricThread = new AsyncLoopThread(uploadMetric); - threads.add(uploadMetricThread); + // SupervisorManger which can shutdown all supervisor and workers return new SupervisorManger(conf, supervisorId, active, threads, @@ -190,6 +188,10 @@ public void killSupervisor(SupervisorManger supervisor) { supervisor.shutdown(); } + private void initShutdownHook(SupervisorManger supervisor) { + Runtime.getRuntime().addShutdownHook(new Thread(supervisor)); + } + private void createPid(Map conf) throws Exception { String pidDir = StormConfig.supervisorPids(conf); @@ -212,6 +214,8 @@ public void run() { supervisorManager = mkSupervisor(conf, null); JStormUtils.redirectOutput("/dev/null"); + + initShutdownHook(supervisorManager); } catch (Exception e) { LOG.error("Failed to start supervisor\n", e); diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/SupervisorManger.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/SupervisorManger.java index 65619067c..dad799668 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/SupervisorManger.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/SupervisorManger.java @@ -12,13 +12,13 @@ import backtype.storm.daemon.Shutdownable; +import com.alibaba.jstorm.callback.AsyncLoopThread; import com.alibaba.jstorm.cluster.DaemonCommon; import com.alibaba.jstorm.cluster.StormClusterState; import com.alibaba.jstorm.cluster.StormConfig; import com.alibaba.jstorm.event.EventManager; import com.alibaba.jstorm.utils.JStormUtils; import com.alibaba.jstorm.utils.PathUtils; -import com.alibaba.jstorm.utils.SmartThread; /** * supervisor shutdown manager which can shutdown supervisor @@ -36,7 +36,7 @@ public class SupervisorManger extends ShutdownWork implements Shutdownable, private AtomicBoolean active; - private Vector threads; + private Vector threads; private EventManager processesEventManager; @@ -51,7 +51,7 @@ public class SupervisorManger extends ShutdownWork implements Shutdownable, private volatile boolean isFinishShutdown = false; public SupervisorManger(Map conf, String supervisorId, - AtomicBoolean active, Vector threads, + AtomicBoolean active, Vector threads, EventManager processesEventManager, EventManager eventManager, Httpserver httpserver, StormClusterState stormClusterState, ConcurrentHashMap workerThreadPidsAtom) { @@ -70,21 +70,23 @@ public SupervisorManger(Map conf, String supervisorId, @Override public void shutdown() { + if (active.getAndSet(false) == false) { + LOG.info("Supervisor has been shutdown before " + supervisorId); + return ; + } LOG.info("Shutting down supervisor " + supervisorId); - active.set(false); - int size = threads.size(); - for (int i = 0; i < size; i++) { - SmartThread thread = threads.elementAt(i); + for (AsyncLoopThread thread : threads) { thread.cleanup(); JStormUtils.sleepMs(10); thread.interrupt(); - try { - thread.join(); - } catch (InterruptedException e) { - LOG.error(e.getMessage(), e); - } +// try { +// thread.join(); +// } catch (InterruptedException e) { +// LOG.error(e.getMessage(), e); +// } + LOG.info("Successfully shutdown thread:" + thread.getThread().getName()); } eventManager.shutdown(); processesEventManager.shutdown(); @@ -94,7 +96,10 @@ public void shutdown() { // TODO Auto-generated catch block LOG.error("Failed to shutdown ZK client", e); } - httpserver.shutdown(); + if (httpserver != null) { + httpserver.shutdown(); + } + // if (this.cgroupManager != null) // try { diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/SyncProcessEvent.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/SyncProcessEvent.java index c70af7bab..b539c12ac 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/SyncProcessEvent.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/SyncProcessEvent.java @@ -13,13 +13,13 @@ import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; +import org.apache.commons.lang.StringUtils; import org.apache.log4j.Logger; import backtype.storm.Config; import backtype.storm.GenericOptionsParser; import backtype.storm.messaging.IContext; import backtype.storm.utils.LocalState; -import backtype.storm.utils.Time; import com.alibaba.jstorm.client.ConfigExtension; import com.alibaba.jstorm.cluster.Common; @@ -32,6 +32,7 @@ import com.alibaba.jstorm.message.zeroMq.MQContext; import com.alibaba.jstorm.task.LocalAssignment; import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.Pair; import com.alibaba.jstorm.utils.PathUtils; import com.alibaba.jstorm.utils.TimeFormat; import com.alibaba.jstorm.utils.TimeUtils; @@ -56,6 +57,12 @@ class SyncProcessEvent extends ShutdownWork { private SandBoxMaker sandBoxMaker; + /** + * Due to the worker startTime is put in Supervisor memory, + * When supervisor restart, the starting worker is likely to be killed + */ + private Map> workerIdToStartTimeAndPort; + // private Supervisor supervisor; /** @@ -85,6 +92,8 @@ public SyncProcessEvent(String supervisorId, Map conf, this.sandBoxMaker = new SandBoxMaker(conf); + this.workerIdToStartTimeAndPort = new HashMap>(); + if (ConfigExtension.isEnableCgroup(conf)) { cgroupManager = new CgroupManager(conf); } @@ -141,6 +150,9 @@ public void run() { */ Set keepPorts = killUselessWorkers(localWorkerStats); + // check new workers + checkNewWorkers(conf); + // start new workers startNewWorkers(keepPorts, localAssignments); @@ -152,63 +164,57 @@ public void run() { } /** - * wait for all workers of the supervisor launch + * mark all new Workers * - * @param conf * @param workerIds - * @throws InterruptedException - * @throws IOException * @pdOid 52b11418-7474-446d-bff5-0ecd68f4954f */ - public void waitForWorkersLaunch(Map conf, Collection workerIds) - throws IOException, InterruptedException { + public void markAllNewWorkers(Map workerIds) { int startTime = TimeUtils.current_time_secs(); - for (String workerId : workerIds) { + for (Entry entry : workerIds.entrySet()) { + + workerIdToStartTimeAndPort.put(entry.getValue(), + new Pair(startTime, entry.getKey())); - waitForWorkerLaunch(conf, workerId, startTime); } } /** - * wait for worker launch if the time is not > * + * check new workers if the time is not > * * SUPERVISOR_WORKER_START_TIMEOUT_SECS, otherwise info failed * * @param conf - * @param workerId - * @param startTime - * @throws IOException - * @throws InterruptedException * @pdOid f0a6ab43-8cd3-44e1-8fd3-015a2ec51c6a */ - public void waitForWorkerLaunch(Map conf, String workerId, int startTime) - throws IOException, InterruptedException { - - LocalState ls = StormConfig.worker_state(conf, workerId); - - while (true) { + public void checkNewWorkers(Map conf) throws IOException, + InterruptedException { + Set workers = new HashSet(); + for (Entry> entry : workerIdToStartTimeAndPort + .entrySet()) { + String workerId = entry.getKey(); + int startTime = entry.getValue().getFirst(); + LocalState ls = StormConfig.worker_state(conf, workerId); WorkerHeartbeat whb = (WorkerHeartbeat) ls .get(Common.LS_WORKER_HEARTBEAT); - if (whb == null - && ((TimeUtils.current_time_secs() - startTime) < JStormUtils - .parseInt(conf - .get(Config.SUPERVISOR_WORKER_START_TIMEOUT_SECS)))) { - LOG.info(workerId + " still hasn't started"); - Time.sleep(500); + if (whb == null) { + if ((TimeUtils.current_time_secs() - startTime) < JStormUtils + .parseInt(conf + .get(Config.SUPERVISOR_WORKER_START_TIMEOUT_SECS))) { + LOG.info(workerId + " still hasn't started"); + } else { + LOG.error("Failed to start Worker " + workerId); + workers.add(workerId); + } } else { - // whb is valid or timeout - break; + LOG.info("Successfully start worker " + workerId); + workers.add(workerId); } } - - WorkerHeartbeat whb = (WorkerHeartbeat) ls - .get(Common.LS_WORKER_HEARTBEAT); - if (whb == null) { - LOG.error("Failed to start Worker " + workerId); - } else { - LOG.info("Successfully start worker " + workerId); + for (String workerId : workers) { + this.workerIdToStartTimeAndPort.remove(workerId); } } @@ -261,7 +267,11 @@ public Map getLocalWorkerStats(Map conf, state = State.timedOut; } else { - state = State.valid; + if (isWorkerDead(workerid)) { + state = State.timedOut; + }else { + state = State.valid; + } } if (state != State.valid) { @@ -397,7 +407,48 @@ public void launchWorker(Map conf, IContext sharedcontext, workerThreadPidsAtom.put(workerId, pid); } + + // filter conflict jar + private Set setFilterJars(Map totalConf) { + Set filterJars = new HashSet(); + + boolean enableClassloader = ConfigExtension.isEnableTopologyClassLoader(totalConf); + if (enableClassloader == false) { + //avoid logback vs log4j conflict + boolean enableLogback = false; + String userDefLogbackConf = ConfigExtension.getUserDefinedLogbackConf(totalConf); + if (StringUtils.isBlank(userDefLogbackConf) == false) { + enableLogback = true; + } + + if (enableLogback == true) { + filterJars.add("slf4j-log4j"); + filterJars.add("log4j"); + + }else { + filterJars.add("log4j-over-slf4j"); + } + } + LOG.info("Remove jars " + filterJars); + return filterJars; + } + + public static boolean isKeyContain(Collection collection, String jar) { + if (collection == null) { + return false; + } + File file = new File(jar); + String fileName = file.getName(); + for(String item : collection) { + + if (fileName.startsWith(item)) { + return true; + } + } + return false; + } + private String getClassPath(String stormjar, String stormHome, Map totalConf) { // String classpath = JStormUtils.current_classpath() + ":" + stormjar; @@ -410,6 +461,9 @@ private String getClassPath(String stormjar, String stormHome, Map totalConf) { Set classSet = new HashSet(); for (String classJar : classpathes) { + if (StringUtils.isBlank(classJar) == true) { + continue; + } classSet.add(classJar); } @@ -434,30 +488,15 @@ private String getClassPath(String stormjar, String stormHome, Map totalConf) { } - // filter jeromq.jar/jzmq.jar to avoid ZMQ.class conflict - String filterJarKeyword = null; - String transport_plugin_klassName = (String) totalConf - .get(Config.STORM_MESSAGING_TRANSPORT); - if (transport_plugin_klassName.equals(MQContext.class - .getCanonicalName())) { - filterJarKeyword = "jeromq"; - } else if (transport_plugin_klassName - .equals("com.alibaba.jstorm.message.jeroMq.JMQContext")) { - filterJarKeyword = "jzmq"; - } + Set filterJars = setFilterJars(totalConf); StringBuilder sb = new StringBuilder(); - if (filterJarKeyword != null) { - for (String jar : classSet) { - if (jar.contains(filterJarKeyword)) { - continue; - } - sb.append(jar + ":"); - } - } else { - for (String jar : classSet) { - sb.append(jar + ":"); + for (String jar : classSet) { + if (isKeyContain(filterJars, jar)) { + LOG.info("Remove " + jar); + continue; } + sb.append(jar + ":"); } if (ConfigExtension.isEnableTopologyClassLoader(totalConf)) { @@ -481,6 +520,42 @@ public String getChildOpts(Map stormConf) { return childopts; } + + public String getLogParameter(Map conf, String stormHome, String topologyName, int port) { + String logFileName = JStormUtils.genLogName( + topologyName, port); + // String logFileName = topologyId + "-worker-" + port + ".log"; + + StringBuilder commandSB = new StringBuilder(); + commandSB.append(" -Dlogfile.name="); + commandSB.append(logFileName); + // commandSB.append(" -Dlog4j.ignoreTCL=true"); + + String userDefLogConf = ConfigExtension.getUserDefinedLog4jConf(conf); + + String logConf = System.getProperty("log4j.configuration"); + + if (StringUtils.isBlank(userDefLogConf) == false) { + LOG.info("Use user fined log4j conf " + userDefLogConf); + commandSB.append(" -Dlog4j.configuration=").append(userDefLogConf); + }else if (StringUtils.isBlank(logConf) == false) { + commandSB.append(" -Dlog4j.configuration=").append(logConf); + }else if (StringUtils.isBlank(stormHome) == false){ + commandSB.append(" -Dlog4j.configuration=File:") + .append(stormHome) + .append(File.separator).append("conf").append(File.separator) + .append("jstorm.log4j.properties"); + }else { + commandSB.append(" -Dlog4j.configuration=File:jstorm.log4j.properties"); + } + + String userDefLogbackConf = ConfigExtension.getUserDefinedLogbackConf(conf); + if (StringUtils.isBlank(userDefLogbackConf) == false) { + commandSB.append(" -Dlogback.configurationFile=").append(userDefLogbackConf); + } + + return commandSB.toString(); + } private String getGcDumpParam(Map totalConf) { // String gcPath = ConfigExtension.getWorkerGcPath(totalConf); @@ -497,7 +572,8 @@ private String getGcDumpParam(Map totalConf) { gc.append("%TOPOLOGYID%-worker-%ID%-"); gc.append(nowStr); gc.append("-gc.log -verbose:gc -XX:HeapDumpPath="); - gc.append(gcPath); + gc.append(gcPath).append(File.separator).append("java-%TOPOLOGYID%-") + .append(nowStr).append(".hprof"); gc.append(" "); return gc.toString(); @@ -550,13 +626,6 @@ public void launchWorker(Map conf, IContext sharedcontext, childopts += getGcDumpParam(totalConf); - childopts = childopts.replace("%ID%", port.toString()); - childopts = childopts.replace("%TOPOLOGYID%", topologyId); - if (stormhome != null) { - childopts = childopts.replace("%JSTORM_HOME%", stormhome); - } else { - childopts = childopts.replace("%JSTORM_HOME%", "./"); - } Map environment = new HashMap(); if (ConfigExtension.getWorkerRedirectOutput(totalConf)) { @@ -565,9 +634,6 @@ public void launchWorker(Map conf, IContext sharedcontext, environment.put("REDIRECT", "false"); } - String logFileName = JStormUtils.genLogName(assignment.getTopologyName(), port); - //String logFileName = topologyId + "-worker-" + port + ".log"; - environment.put("LD_LIBRARY_PATH", (String) totalConf.get(Config.JAVA_LIBRARY_PATH)); @@ -597,23 +663,12 @@ public void launchWorker(Map conf, IContext sharedcontext, commandSB.append(" -Djava.library.path="); commandSB.append((String) totalConf.get(Config.JAVA_LIBRARY_PATH)); - commandSB.append(" -Dlogfile.name="); - commandSB.append(logFileName); - - // commandSB.append(" -Dlog4j.ignoreTCL=true"); - if (stormhome != null) { - // commandSB.append(" -Dlogback.configurationFile=" + stormhome + - // "/conf/cluster.xml"); - commandSB.append(" -Dlog4j.configuration=File:" + stormhome - + "/conf/jstorm.log4j.properties"); commandSB.append(" -Djstorm.home="); commandSB.append(stormhome); - } else { - // commandSB.append(" -Dlogback.configurationFile=cluster.xml"); - commandSB - .append(" -Dlog4j.configuration=File:jstorm.log4j.properties"); - } + } + + commandSB.append(getLogParameter(totalConf, stormhome, assignment.getTopologyName(), port)); String classpath = getClassPath(stormjar, stormhome, totalConf); String workerClassPath = (String) totalConf @@ -658,10 +713,19 @@ public void launchWorker(Map conf, IContext sharedcontext, commandSB.append(" "); commandSB.append(workerClassPath + ":" + stormjar); - LOG.info("Launching worker with command: " + commandSB); + String cmd = commandSB.toString(); + cmd = cmd.replace("%ID%", port.toString()); + cmd = cmd.replace("%TOPOLOGYID%", topologyId); + if (stormhome != null) { + cmd = cmd.replace("%JSTORM_HOME%", stormhome); + } else { + cmd = cmd.replace("%JSTORM_HOME%", "./"); + } + + LOG.info("Launching worker with command: " + cmd); LOG.info("Environment:" + environment.toString()); - JStormUtils.launch_process(commandSB.toString(), environment, true); + JStormUtils.launch_process(cmd, environment, true); } private Set killUselessWorkers( @@ -673,17 +737,20 @@ private Set killUselessWorkers( String workerid = entry.getKey(); StateHeartbeat hbstate = entry.getValue(); + if (workerIdToStartTimeAndPort.containsKey(workerid)) + continue; if (hbstate.getState().equals(State.valid)) { // hbstate.getHeartbeat() won't be null keepPorts.add(hbstate.getHeartbeat().getPort()); } else { if (hbstate.getHeartbeat() != null) { - removed.put(workerid, hbstate.getHeartbeat().getTopologyId()); - }else { + removed.put(workerid, hbstate.getHeartbeat() + .getTopologyId()); + } else { removed.put(workerid, null); } - + StringBuilder sb = new StringBuilder(); sb.append("Shutting down and clearing state for id "); sb.append(workerid); @@ -693,13 +760,17 @@ private Set killUselessWorkers( LOG.info(sb); } } - - shutWorker(conf, supervisorId, removed, workerThreadPids, cgroupManager); + shutWorker(conf, supervisorId, removed, workerThreadPids, cgroupManager); for (String removedWorkerId : removed.keySet()) { localWorkerStats.remove(removedWorkerId); } + // new workers need be keep + for (Entry> entry : workerIdToStartTimeAndPort + .entrySet()) { + keepPorts.add(entry.getValue().getSecond()); + } return keepPorts; } @@ -771,14 +842,41 @@ private void startNewWorkers(Set keepPorts, * FIXME, workerIds should be Set, not Collection, but here simplify the * logic */ - Collection workerIds = newWorkerIds.values(); + markAllNewWorkers(newWorkerIds); + // try { + // waitForWorkersLaunch(conf, workerIds); + // } catch (IOException e) { + // LOG.error(e + " waitForWorkersLaunch failed"); + // } catch (InterruptedException e) { + // LOG.error(e + " waitForWorkersLaunch failed"); + // } + } + + boolean isWorkerDead(String workerId) { + try { - waitForWorkersLaunch(conf, workerIds); + List pids = getPid(conf, workerId); + if (pids == null || pids.size() == 0) { + //local mode doesn't exist pid + return false; + } + // if all pid in pids are dead, then the worker is dead + for (String pid : pids) { + boolean isDead = JStormUtils.isProcDead(pid); + if (isDead == true) { + LOG.info("Found " + workerId + " is dead " ); + }else { + return false; + } + } + + return true; } catch (IOException e) { - LOG.error(e + " waitForWorkersLaunch failed"); - } catch (InterruptedException e) { - LOG.error(e + " waitForWorkersLaunch failed"); + LOG.info("Failed to check whether worker is dead through /proc/pid", e); + return false; } + + } } diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/RefreshConnections.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/RefreshConnections.java index 3f9ed761b..11b9a4b03 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/RefreshConnections.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/RefreshConnections.java @@ -20,6 +20,9 @@ import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; import com.alibaba.jstorm.task.Assignment; import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.TimeUtils; +import com.alibaba.jstorm.task.TaskInfo; +import com.alibaba.jstorm.task.heartbeat.TaskHeartbeat; /** * @@ -55,6 +58,8 @@ public class RefreshConnections extends RunnableCallback { private Integer frequence; private String supervisorId; + + private int taskTimeoutSecs; // private ReentrantReadWriteLock endpoint_socket_lock; @@ -76,6 +81,9 @@ public RefreshConnections(WorkerData workerData, Set outbound_tasks) { // this.endpoint_socket_lock = endpoint_socket_lock; frequence = JStormUtils.parseInt( conf.get(Config.TASK_REFRESH_POLL_SECS), 5); + + taskTimeoutSecs = JStormUtils.parseInt(conf.get(Config.TASK_HEARTBEAT_FREQUENCY_SECS), 10); + taskTimeoutSecs = taskTimeoutSecs*3; } @Override @@ -170,6 +178,19 @@ public void run() { LOG.info("Remove connection to " + node_port); nodeportSocket.remove(node_port).close(); } + + // Update the status of all outbound tasks + for (Integer taskId : outboundTasks) { + boolean isActive = false; + int currentTime = TimeUtils.current_time_secs(); + TaskHeartbeat tHB = zkCluster.task_heartbeat(topologyId, taskId); + if (tHB != null) { + int taskReportTime = tHB.getTimeSecs(); + if ((currentTime - taskReportTime) < taskTimeoutSecs) + isActive = true; + } + workerData.updateOutboundTaskStatus(taskId, isActive); + } } } catch (Exception e) { LOG.error("Failed to refresh worker Connection", e); diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/TimeTick.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/TimeTick.java deleted file mode 100644 index 15183ca60..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/TimeTick.java +++ /dev/null @@ -1,108 +0,0 @@ -package com.alibaba.jstorm.daemon.worker; - -import java.util.HashMap; -import java.util.Map; -import java.util.Map.Entry; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.log4j.Logger; - -import backtype.storm.Config; -import backtype.storm.utils.DisruptorQueue; - -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.task.acker.Acker; -import com.alibaba.jstorm.utils.JStormUtils; - -/** - * Timely check whether topology is active or not from ZK - * - * @author yannian/Longda - * - */ -public class TimeTick extends RunnableCallback { - private static Logger LOG = Logger.getLogger(TimeTick.class); - - - private AtomicBoolean active; - - private Integer frequence; - private Integer firstSleep; - - private static Map queues = new HashMap(); - - public static void registerTimer(String name, DisruptorQueue queue) { - queues.put(name, queue); - } - - public static class Tick { - private final long time; - private final String name; - - public Tick(String name) { - this.name = name; - time = System.currentTimeMillis(); - } - - public long getTime() { - return time; - } - - public String getName() { - return name; - } - - } - - @SuppressWarnings("rawtypes") - public TimeTick(WorkerData workerData) { - active = workerData.getActive(); - - Map conf = workerData.getStormConf(); - int msgTimeOut = JStormUtils.parseInt( - conf.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS), 30); - frequence = (msgTimeOut) / (Acker.TIMEOUT_BUCKET_NUM - 1); - if (frequence <= 0) { - frequence = 1; - } - - firstSleep = JStormUtils.parseInt( - conf.get(Config.SUPERVISOR_WORKER_START_TIMEOUT_SECS), 120); - - firstSleep += frequence; - LOG.info("TimeTick frequence " + frequence); - } - - private boolean isFirstTime = true; - - @Override - public void run() { - - if (active.get() == false) { - return; - } - - if (isFirstTime == true) { - isFirstTime = false; - JStormUtils.sleepMs(firstSleep * 1000); - LOG.info("Start TimeTick"); - } - - for (Entry entry: queues.entrySet()) { - String name = entry.getKey(); - DisruptorQueue queue = entry.getValue(); - Tick tick = new Tick(name); - queue.publish(tick); - LOG.debug("Issue time tick " + name); - } - - } - - @Override - public Object getResult() { - if (active.get()) { - return frequence; - } - return -1; - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/VirtualPortDispatch.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/VirtualPortDispatch.java index c363d3eff..e01abaeb5 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/VirtualPortDispatch.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/VirtualPortDispatch.java @@ -59,6 +59,7 @@ public void cleanup() { } recvConnection = null; + Metrics.unregister(null, MetricDef.DISPATCH_QUEUE, null, Metrics.MetricType.WORKER); LOG.info("Successfully shudown VirtualPortDispatch"); } diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/Worker.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/Worker.java index 1aeb7d4f8..e80579e26 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/Worker.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/Worker.java @@ -104,6 +104,8 @@ private RefreshConnections makeRefreshConnections() { // get output streams of every task Set outboundTasks = worker_output_tasks(); + + workerData.initOutboundTaskStatus(outboundTasks); RefreshConnections refresh_connections = new RefreshConnections( workerData, outboundTasks); @@ -169,10 +171,6 @@ public WorkerShutdown execute() throws Exception { Thread.MIN_PRIORITY, true); threads.add(refreshconn); - TimeTick timeTick = new TimeTick(workerData); - AsyncLoopThread tick = new AsyncLoopThread(timeTick); - threads.add(tick); - // refresh ZK active status RefreshActive refreshZkActive = new RefreshActive(workerData); AsyncLoopThread refreshzk = new AsyncLoopThread(refreshZkActive, false, @@ -443,26 +441,26 @@ public static void main(String[] args) { System.exit(-1); } - String topology_id = args[0]; - String supervisor_id = args[1]; - String port_str = args[2]; - String worker_id = args[3]; - String jar_path = args[4]; - - killOldWorker(port_str); + StringBuilder sb = new StringBuilder(); - Map conf = Utils.readStormConfig(); - StormConfig.validate_distributed_mode(conf); + try { + String topology_id = args[0]; + String supervisor_id = args[1]; + String port_str = args[2]; + String worker_id = args[3]; + String jar_path = args[4]; - JStormServerUtils.startTaobaoJvmMonitor(); + killOldWorker(port_str); - StringBuilder sb = new StringBuilder(); - sb.append("topologyId:" + topology_id + ", "); - sb.append("port:" + port_str + ", "); - sb.append("workerId:" + worker_id + ", "); - sb.append("jar_path:" + jar_path + "\n"); + Map conf = Utils.readStormConfig(); + StormConfig.validate_distributed_mode(conf); - try { + JStormServerUtils.startTaobaoJvmMonitor(); + + sb.append("topologyId:" + topology_id + ", "); + sb.append("port:" + port_str + ", "); + sb.append("workerId:" + worker_id + ", "); + sb.append("jar_path:" + jar_path + "\n"); WorkerShutdown sd = mk_worker(conf, null, topology_id, supervisor_id, Integer.parseInt(port_str), worker_id, diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerData.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerData.java index a75fedba9..bd2bc4612 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerData.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerData.java @@ -10,6 +10,8 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.commons.lang.StringUtils; @@ -33,8 +35,10 @@ import com.alibaba.jstorm.cluster.StormConfig; import com.alibaba.jstorm.daemon.nimbus.StatusType; import com.alibaba.jstorm.daemon.worker.metrics.MetricReporter; +import com.alibaba.jstorm.daemon.worker.timer.TimerTrigger; import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; import com.alibaba.jstorm.task.Assignment; +import com.alibaba.jstorm.task.TaskInfo; import com.alibaba.jstorm.task.TaskShutdownDameon; import com.alibaba.jstorm.utils.JStormServerUtils; import com.alibaba.jstorm.utils.JStormUtils; @@ -114,6 +118,11 @@ public class WorkerData { private List shutdownTasks; private MetricReporter metricReporter; + + private Map outTaskStatus; //true => active + + public static final int THREAD_POOL_NUM = 4; + private ScheduledExecutorService threadPool; @SuppressWarnings({ "rawtypes", "unchecked" }) public WorkerData(Map conf, IContext context, String topology_id, @@ -148,6 +157,18 @@ public WorkerData(Map conf, IContext context, String topology_id, LOG.info("Worker Configuration " + stormConf); try { + + boolean enableClassloader = ConfigExtension + .isEnableTopologyClassLoader(stormConf); + boolean enableDebugClassloader = ConfigExtension + .isEnableClassloaderDebug(stormConf); + + if (jar_path == null && enableClassloader == true) { + LOG.error("enable classloader, but not app jar"); + throw new InvalidParameterException(); + } + + URL[] urlArray = new URL[0]; if (jar_path != null) { String[] paths = jar_path.split(":"); Set urls = new HashSet(); @@ -157,16 +178,13 @@ public WorkerData(Map conf, IContext context, String topology_id, URL url = new URL("File:" + path); urls.add(url); } - WorkerClassLoader.mkInstance(urls.toArray(new URL[0]), - ClassLoader.getSystemClassLoader(), ClassLoader - .getSystemClassLoader().getParent(), - ConfigExtension.isEnableTopologyClassLoader(stormConf)); - } else { - WorkerClassLoader.mkInstance(new URL[0], ClassLoader - .getSystemClassLoader(), ClassLoader - .getSystemClassLoader().getParent(), ConfigExtension - .isEnableTopologyClassLoader(stormConf)); + urlArray = urls.toArray(new URL[0]); + } + + WorkerClassLoader.mkInstance(urlArray, ClassLoader + .getSystemClassLoader(), ClassLoader.getSystemClassLoader() + .getParent(), enableClassloader, enableDebugClassloader); } catch (Exception e) { // TODO Auto-generated catch block LOG.error("init jarClassLoader error!", e); @@ -230,6 +248,11 @@ public WorkerData(Map conf, IContext context, String topology_id, contextMaker = new ContextMaker(this); metricReporter = new MetricReporter(this); + + outTaskStatus = new HashMap(); + + threadPool = Executors.newScheduledThreadPool(THREAD_POOL_NUM); + TimerTrigger.setScheduledExecutorService(threadPool); LOG.info("Successfully create WorkerData"); @@ -415,4 +438,22 @@ public void setMetricsReporter(MetricReporter reporter) { public MetricReporter getMetricsReporter() { return this.metricReporter; } + + public void initOutboundTaskStatus(Set outboundTasks) { + for (Integer taskId : outboundTasks) { + outTaskStatus.put(taskId, false); + } + } + + public void updateOutboundTaskStatus(Integer taskId, boolean isActive) { + outTaskStatus.put(taskId, isActive); + } + + public boolean isOutboundTaskActive(Integer taskId) { + return outTaskStatus.get(taskId) != null ? outTaskStatus.get(taskId) : false; + } + + public ScheduledExecutorService getThreadPool() { + return threadPool; + } } diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerMetricInfo.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerMetricInfo.java index 60b3d5bcc..f19a9c328 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerMetricInfo.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerMetricInfo.java @@ -1,22 +1,23 @@ package com.alibaba.jstorm.daemon.worker; -import java.util.Map; -import java.util.HashMap; import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; import org.apache.log4j.Logger; -import com.codahale.metrics.Metric; -import com.codahale.metrics.Gauge; -import com.codahale.metrics.Timer; +import com.alibaba.jstorm.metric.MetricInfo; +import com.alibaba.jstorm.metric.Metrics.QueueGauge; +import com.alibaba.jstorm.utils.JStormUtils; import com.codahale.metrics.Counter; +import com.codahale.metrics.Gauge; import com.codahale.metrics.Histogram; import com.codahale.metrics.Meter; +import com.codahale.metrics.Metric; import com.codahale.metrics.Snapshot; -import com.esotericsoftware.minlog.Log; -import com.alibaba.jstorm.metric.MetricInfo; -import com.alibaba.jstorm.metric.Metrics.QueueGauge; -import com.alibaba.jstorm.utils.JStormUtils; +import com.codahale.metrics.Timer; /** @@ -146,4 +147,10 @@ private void sum(Map dataMap, String name, Double value) { value = JStormUtils.formatDoubleDecPoint4(value); dataMap.put(name, value); } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } } \ No newline at end of file diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerShutdown.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerShutdown.java index 84179a20d..c2d2e216c 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerShutdown.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerShutdown.java @@ -2,6 +2,7 @@ import java.util.List; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.log4j.Logger; @@ -37,6 +38,7 @@ public class WorkerShutdown implements ShutdownableDameon { private StormClusterState zkCluster; private ClusterState cluster_state; private MetricReporter metricReporter; + private ScheduledExecutorService threadPool; // active nodeportSocket context zkCluster zkClusterstate public WorkerShutdown(WorkerData workerData, @@ -51,6 +53,7 @@ public WorkerShutdown(WorkerData workerData, this.context = workerData.getContext(); this.zkCluster = workerData.getZkCluster(); this.cluster_state = workerData.getZkClusterstate(); + this.threadPool = workerData.getThreadPool(); this.metricReporter = metricReporter; Runtime.getRuntime().addShutdownHook(new Thread(this)); @@ -64,8 +67,12 @@ public WorkerShutdown(WorkerData workerData, @Override public void shutdown() { - active.set(false); + if (active.getAndSet(false) == false) { + LOG.info("Worker has been shutdown already"); + return; + } + threadPool.shutdown(); metricReporter.shutdown(); // shutdown tasks diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/MetricKVMsg.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/MetricKVMsg.java index 283492522..abaf59710 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/MetricKVMsg.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/MetricKVMsg.java @@ -1,14 +1,19 @@ package com.alibaba.jstorm.daemon.worker.metrics; import java.util.HashMap; +import java.util.List; +import java.util.ArrayList; import java.util.Map; import java.util.Map.Entry; import java.util.concurrent.TimeUnit; import org.apache.log4j.Logger; +import com.alibaba.jstorm.cluster.StormClusterState; +import com.alibaba.jstorm.metric.UserDefMetricData; import com.alibaba.jstorm.metric.metrdata.*; -import com.esotericsoftware.minlog.Log; +import backtype.storm.generated.UserDefMetric; + //count metric data,and transform metric_data to Alimonitor message public class MetricKVMsg { private static final Logger LOG = Logger.getLogger(MetricKVMsg.class); @@ -19,7 +24,7 @@ public enum MetricType{ private Map gaugeMapKV = new HashMap(); //count value of Gauge private Map counterMapKV = new HashMap(); //count value of Counter - private Map> histogramMapKV = new HashMap>(); // count data of Histogram + private Map> histogramMapKV = new HashMap>(); // count data of Histogram private Map> timerMapKV = new HashMap>(); // count data of Timer private Map> meterMapKV = new HashMap>(); // count data of Meter private Map countMap = new HashMap(); @@ -49,11 +54,11 @@ public void addCounterToKVMap(Map kVMap) { } public void addHistogramToKVMap(Map kVMap) { - for (Entry> entry : histogramMapKV.entrySet()) { + for (Entry> entry : histogramMapKV.entrySet()) { String name = entry.getKey(); - Map typeMap = entry.getValue(); + Map typeMap = entry.getValue(); - for (Entry typeEntry : typeMap.entrySet()) { + for (Entry typeEntry : typeMap.entrySet()) { kVMap.put(name+ "_" + typeEntry.getKey().toString(), typeEntry.getValue()); } } @@ -112,19 +117,19 @@ public void countHistogramMetric(Map histogramMap){ String taskMetricName = entry.getKey(); String userDefName = taskMetricName.substring(taskMetricName.indexOf(":") + 1); - long maxValue = entry.getValue().getMax(); - long minValue = entry.getValue().getMin(); - long meanValue = (long)entry.getValue().getMean(); + double maxValue = entry.getValue().getMax(); + double minValue = entry.getValue().getMin(); + double meanValue = entry.getValue().getMean(); - Map temMap = histogramMapKV.get(userDefName); + Map temMap = histogramMapKV.get(userDefName); if(temMap == null){ - temMap = new HashMap(); + temMap = new HashMap(); histogramMapKV.put(userDefName, temMap); } maxValue += (temMap.get(MetricType.max) == null ? 0l : temMap.get(MetricType.max)); - minValue += (temMap.get(MetricType.min) == null ? 0l : temMap.get(MetricType.max)); - meanValue += (temMap.get(MetricType.mean) == null ? 0l : temMap.get(MetricType.mean)); + minValue += (temMap.get(MetricType.min) == null ? 0l : temMap.get(MetricType.min)); + meanValue += (temMap.get(MetricType.mean) == null ? 0.0 : temMap.get(MetricType.mean)); temMap.put(MetricType.max, maxValue); temMap.put(MetricType.min, minValue); @@ -145,12 +150,16 @@ public void countTimerMetric(Map timerMap){ timerMapKV.put(userDefName, temMap); } + // It is possible that the timer function is not called in some tasks. So, for + // this case, the value should not be involved for following average time calculation. + if (meanValue != 0.0) { + Integer count = (countMap.get(userDefName) == null ? 0 : countMap.get(userDefName)); + count++; + countMap.put(userDefName, count); + } + meanValue += (temMap.get(MetricType.mean) == null ? 0.0 : temMap.get(MetricType.mean)); - temMap.put(MetricType.mean, meanValue); - - Integer count = (countMap.get(userDefName) == null ? 0 : countMap.get(userDefName)); - count++; - countMap.put(userDefName, count); + temMap.put(MetricType.mean, meanValue); } } @@ -159,8 +168,8 @@ public void calcAvgTimer() { String userDefName = entry.getKey(); Map valueMap = entry.getValue(); Integer count = countMap.get(userDefName); - if (count == null) { - Log.warn("Name=" + userDefName + " is not found in countMap for timer."); + if (count == null || count == 0) { + LOG.warn("Name=" + userDefName + " is not found in countMap for timer, or count is zero"); continue; } double meanValue = (valueMap.get(MetricType.mean))/count; @@ -198,4 +207,48 @@ public void emptyCountMap() { private double convertDurationFromNsToMs(double duration) { return duration / TimeUnit.MILLISECONDS.toNanos(1); } + + public static MetricKVMsg getMetricKVMsg(String topologyId, StormClusterState clusterState) throws Exception { + List workerIds = clusterState.monitor_user_workers(topologyId); + MetricKVMsg topologyMetricMsg = new MetricKVMsg(); + for (String workerId : workerIds) { + UserDefMetricData useWorkDefMetric = clusterState.get_userDef_metric(topologyId, workerId); + //add metric based on worker to useWorkDefMetric + topologyMetricMsg.countGangeMetric(useWorkDefMetric.getGaugeDataMap()); + topologyMetricMsg.countCounterMetric(useWorkDefMetric.getCounterDataMap()); + topologyMetricMsg.countHistogramMetric(useWorkDefMetric.getHistogramDataMap()); + topologyMetricMsg.countTimerMetric(useWorkDefMetric.getTimerDataMap()); + topologyMetricMsg.countMeterMetric(useWorkDefMetric.getMeterDataMap()); + } + topologyMetricMsg.calcAvgTimer(); + topologyMetricMsg.emptyCountMap(); + + return topologyMetricMsg; + } + + public List convertToUserDefMetric() { + List userDefMetrics = new ArrayList(); + + for (Entry entry : gaugeMapKV.entrySet()) { + userDefMetrics.add(new UserDefMetric("Gauge", entry.getKey(), entry.getValue())); + } + + for (Entry entry : counterMapKV.entrySet()) { + userDefMetrics.add(new UserDefMetric("Counter", entry.getKey(), entry.getValue())); + } + + for (Entry> entry : histogramMapKV.entrySet()) { + userDefMetrics.add(new UserDefMetric("Histogram", entry.getKey(), entry.getValue().get(MetricType.mean))); + } + + for (Entry> entry : timerMapKV.entrySet()) { + userDefMetrics.add(new UserDefMetric("Timer", entry.getKey(), entry.getValue().get(MetricType.mean))); + } + + for (Entry> entry : meterMapKV.entrySet()) { + userDefMetrics.add(new UserDefMetric("Meter", entry.getKey(), entry.getValue().get(MetricType.mean))); + } + + return userDefMetrics; + } } diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/StormMetricReporter.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/StormMetricReporter.java index 821ac9a4d..c541ece14 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/StormMetricReporter.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/StormMetricReporter.java @@ -36,6 +36,7 @@ import com.alibaba.jstorm.task.TaskMetricInfo; import com.alibaba.jstorm.utils.JStormServerUtils; import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.NetWorkUtils; public class StormMetricReporter extends ScheduledReporter { /** @@ -312,10 +313,13 @@ private void updateTaskMetricsToZK(Map> metricMap, bool List MetricList = entry.getValue(); try { - String component = clusterState.task_info(topologyId, Integer.valueOf(taskId)).getComponentId(); + String component = workerData.getTasksToComponent().get(Integer.valueOf(taskId)); TaskMetricInfo taskMetricInfo = new TaskMetricInfo(taskId, component); for(MetricInfo metricInfo : MetricList) { + if(metricPerf == false && ((metricInfo.getMetric() instanceof Timer) || + (metricInfo.getMetric() instanceof Histogram))) + continue; taskMetricInfo.updateMetricData(metricInfo); } @@ -364,13 +368,16 @@ private void updateWorkerMetricsToZK(List metricList, boolean metric String topologyId = workerData.getTopologyId(); String hostName; - hostName = JStormServerUtils.getHostName(workerData.getConf()); + hostName = NetWorkUtils.ip(); String workerId = hostName + ":" + workerData.getPort(); WorkerMetricInfo workerMetricInfo = new WorkerMetricInfo(hostName, workerData.getPort()); try { //Set metrics data - for(MetricInfo metricInfo : metricList) { + for(MetricInfo metricInfo : metricList) { + if(metricPerf == false && ((metricInfo.getMetric() instanceof Timer) || + (metricInfo.getMetric() instanceof Histogram))) + continue; workerMetricInfo.updateMetricData(metricInfo); } @@ -401,9 +408,9 @@ private void updateUserDefMetricsToZK(boolean metricPerf) { userDefMetricData.updateFromMeterData(userDefMetric.getMeter()); // If metrics performance is disable, Timer & Histogram metrics will not be monitored, // and the corresponding metrics data will not be sent to ZK either. - if (metricPerf == false) { - userDefMetricData.updateFromHistogramData(userDefMetric.getHistogram()); - userDefMetricData.updateFromTimerData(userDefMetric.getTimer()); + if (metricPerf) { + userDefMetricData.updateFromHistogramData(userDefMetric.getHistogram()); + userDefMetricData.updateFromTimerData(userDefMetric.getTimer()); } try { @@ -417,7 +424,7 @@ private void updateUserDefMetricsToZK(boolean metricPerf) { doCallback(userDefMetric.getGauge()); doCallback(userDefMetric.getCounter()); doCallback(userDefMetric.getMeter()); - if (metricPerf == false) { + if (metricPerf) { doCallback(userDefMetric.getHistogram()); doCallback(userDefMetric.getTimer()); } diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/UploadMetricFromZK.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/UploadMetricFromZK.java index 1c6f43e50..6409834b6 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/UploadMetricFromZK.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/UploadMetricFromZK.java @@ -122,22 +122,7 @@ public void uploadUseDefMetric(StormClusterState clusterState) { Map totalMsg = new HashMap(); for (String topologyId : active_topologys) { - Map> compont_metrics = new HashMap>(); - List workerIds = clusterState.monitor_user_workers(topologyId); - if(workerIds == null) - continue; - MetricKVMsg topologyMetricMsg = new MetricKVMsg(); - for(String workerId : workerIds) { - UserDefMetricData useWorkDefMetric = clusterState.get_userDef_metric(topologyId, workerId); - //add metric based on worker to useWorkDefMetric - topologyMetricMsg.countGangeMetric(useWorkDefMetric.getGaugeDataMap()); - topologyMetricMsg.countCounterMetric(useWorkDefMetric.getCounterDataMap()); - topologyMetricMsg.countHistogramMetric(useWorkDefMetric.getHistogramDataMap()); - topologyMetricMsg.countTimerMetric(useWorkDefMetric.getTimerDataMap()); - topologyMetricMsg.countMeterMetric(useWorkDefMetric.getMeterDataMap()); - } - topologyMetricMsg.calcAvgTimer(); - topologyMetricMsg.emptyCountMap(); + MetricKVMsg topologyMetricMsg = MetricKVMsg.getMetricKVMsg(topologyId, clusterState); Map ret = topologyMetricMsg.convertToKVMap(); if(ret.size() >0) totalMsg.putAll(ret); } diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/UploadSupervMetric.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/UploadSupervMetric.java index cdb6c41de..0b09f2c22 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/UploadSupervMetric.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/UploadSupervMetric.java @@ -22,6 +22,7 @@ import com.alibaba.jstorm.metric.MetricDef; import com.alibaba.jstorm.task.TaskMetricInfo; import com.alibaba.jstorm.task.Assignment; +import com.alibaba.jstorm.utils.JStormUtils; public class UploadSupervMetric extends RunnableCallback { private static Logger LOG = Logger.getLogger(UploadSupervMetric.class); @@ -160,6 +161,8 @@ public void buildTaskJsonMsg(String topologyId, Set taskSet, boolean me taskKV.putAll(taskMetric.getHistogramData()); } + taskKV.put("Task_Error_Info", getTaskErrInfo(topologyId, taskId)); + jsonMsgTasks.add(taskKV); } catch (Exception e) { LOG.error("Failed to buildTaskJsonMsg, taskID=" + taskId + ", e=" + e); @@ -198,6 +201,34 @@ public void buildWorkerJsonMsg(String topologyId, Set workerSet, boolea } } } + + public String getTaskErrInfo(String topologyId, int taskId) { + String ret = null; + long currTime = System.currentTimeMillis()/1000; + + try { + List errorTimeStamps = cluster.task_error_time(topologyId, taskId); + + // Only send the errors which ocurr during last min + for (String time : errorTimeStamps) { + long errTime = JStormUtils.parseLong(time); + if (currTime - errTime < ConfigExtension.getTaskErrorReportInterval(conf)) { + String errInfo = cluster.task_error_info(topologyId, taskId, errTime); + if (errInfo.indexOf("queue is full") == -1) { + if (ret == null) + ret = errInfo; + else + ret = ret + "; " + errInfo; + } + } + } + } catch (Exception e) { + LOG.error("Failed to read task error info for topo=" + topologyId + ", taskId=" + taskId + + ". Cause is \"" + e.getMessage() + "\""); + } + + return ret; + } public void clean() { } diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/timer/RotatingMapTrigger.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/timer/RotatingMapTrigger.java new file mode 100644 index 000000000..0cbcfc33b --- /dev/null +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/timer/RotatingMapTrigger.java @@ -0,0 +1,64 @@ +package com.alibaba.jstorm.daemon.worker.timer; + +import java.util.Map; + +import org.apache.log4j.Logger; + +import backtype.storm.Config; +import backtype.storm.utils.DisruptorQueue; + +import com.alibaba.jstorm.task.acker.Acker; +import com.alibaba.jstorm.utils.JStormUtils; + +public class RotatingMapTrigger extends TimerTrigger { + private static final Logger LOG = Logger + .getLogger(RotatingMapTrigger.class); + + public RotatingMapTrigger(Map conf, String name, DisruptorQueue queue) { + this.name = name; + this.queue = queue; + + int msgTimeOut = JStormUtils.parseInt( + conf.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS), 30); + frequence = (msgTimeOut) / (Acker.TIMEOUT_BUCKET_NUM - 1); + if (frequence <= 0) { + frequence = 1; + } + + firstTime = JStormUtils.parseInt( + conf.get(Config.SUPERVISOR_WORKER_START_TIMEOUT_SECS), 120); + + firstTime += frequence; + } + + @Override + public void updateObject() { + this.object = new Tick(name); + } + + public static final String ROTATINGMAP_STREAMID = "__rotating_tick"; + + // In fact, RotatingMapTrigger can use TickTuple, + // which set the stream ID is ROTATINGMAP_STREAMID + // But in order to improve performance, JStorm use RotatingMapTrigger.Tick + + public static class Tick { + private final long time; + private final String name; + + public Tick(String name) { + this.name = name; + time = System.currentTimeMillis(); + } + + public long getTime() { + return time; + } + + public String getName() { + return name; + } + + } + +} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/timer/TickTupleTrigger.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/timer/TickTupleTrigger.java new file mode 100644 index 000000000..4394e3d8f --- /dev/null +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/timer/TickTupleTrigger.java @@ -0,0 +1,39 @@ +package com.alibaba.jstorm.daemon.worker.timer; + +import org.apache.log4j.Logger; + +import backtype.storm.Constants; +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.TupleImplExt; +import backtype.storm.tuple.Values; +import backtype.storm.utils.DisruptorQueue; + +import com.alibaba.jstorm.utils.TimeUtils; + +public class TickTupleTrigger extends TimerTrigger { + private static final Logger LOG = Logger.getLogger(TickTupleTrigger.class); + + TopologyContext topologyContext; + + public TickTupleTrigger(TopologyContext topologyContext, int frequence, + String name, DisruptorQueue queue) { + this.name = name; + this.queue = queue; + if (frequence <= 0) { + LOG.warn(" The frequence of " + name + " is invalid"); + frequence = 1; + } + this.firstTime = frequence; + this.frequence = frequence; + this.topologyContext = topologyContext; + + } + + @Override + public void updateObject() { + this.object = new TupleImplExt(topologyContext, new Values( + TimeUtils.current_time_secs()), (int) Constants.SYSTEM_TASK_ID, + Constants.SYSTEM_TICK_STREAM_ID); + } + +} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/timer/TimerTrigger.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/timer/TimerTrigger.java new file mode 100644 index 000000000..c6e40e02b --- /dev/null +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/timer/TimerTrigger.java @@ -0,0 +1,118 @@ +package com.alibaba.jstorm.daemon.worker.timer; + +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; +import org.apache.log4j.Logger; + +import backtype.storm.utils.DisruptorQueue; + +import com.lmax.disruptor.InsufficientCapacityException; + +public class TimerTrigger implements Runnable { + private static final Logger LOG = Logger.getLogger(TimerTrigger.class); + + private static ScheduledExecutorService threadPool; + + public static void setScheduledExecutorService( + ScheduledExecutorService scheduledExecutorService) { + threadPool = scheduledExecutorService; + } + + protected String name; + protected int firstTime; + protected int frequence; + protected DisruptorQueue queue; + protected Object object; + protected boolean block = true; + + public void register() { + threadPool.scheduleAtFixedRate(this, firstTime, frequence, + TimeUnit.SECONDS); + LOG.info("Successfully register timer " + this); + } + + public void updateObject() { + + } + + @Override + public void run() { + + try { + updateObject(); + + if (object == null) { + LOG.info("Timer " + name + " 's object is null "); + return; + } + queue.publish(object, block); + }catch(InsufficientCapacityException e) { + LOG.warn("Failed to public timer event to " + name); + return; + }catch (Exception e) { + LOG.warn("Failed to public timer event to " + name, e); + return; + } + + LOG.debug(" Trigger timer event to " + name); + + } + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public int getFirstTime() { + return firstTime; + } + + public void setFirstTime(int firstTime) { + this.firstTime = firstTime; + } + + public int getFrequence() { + return frequence; + } + + public void setFrequence(int frequence) { + this.frequence = frequence; + } + + public DisruptorQueue getQueue() { + return queue; + } + + public void setQueue(DisruptorQueue queue) { + this.queue = queue; + } + + public Object getObject() { + return object; + } + + public void setObject(Object object) { + this.object = object; + } + + public boolean isBlock() { + return block; + } + + public void setBlock(boolean block) { + this.block = block; + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } + +} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/ReconnectRunnable.java b/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/ReconnectRunnable.java index 778c53f0f..d155aff13 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/ReconnectRunnable.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/ReconnectRunnable.java @@ -18,13 +18,14 @@ public void pushEvent(NettyClient client) { } private boolean closed = false; + private Thread thread = null; @Override public void run() { LOG.info("Successfully start reconnect thread"); - + thread = Thread.currentThread(); while(closed == false) { NettyClient client = null; try { @@ -44,6 +45,7 @@ public void run() { @Override public void shutdown() { closed = true; + thread.interrupt(); } @Override diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/FollowerRunnable.java b/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/FollowerRunnable.java index 22ffcb1b0..34bfa69f4 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/FollowerRunnable.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/FollowerRunnable.java @@ -8,6 +8,7 @@ import java.util.UUID; import org.apache.commons.io.FileUtils; +import org.apache.commons.lang.StringUtils; import org.apache.log4j.Logger; import org.apache.thrift7.TException; @@ -76,27 +77,51 @@ public void run() { } }; } + + public boolean isLeader(String zkMaster) { + if (StringUtils.isBlank(zkMaster) == true) { + return false; + } + + if (hostPort.equalsIgnoreCase(zkMaster) == true) { + return true; + } + + String[] part = zkMaster.split(":"); + return NetWorkUtils.equals(part[0], NetWorkUtils.ip()) ; + } @Override public void run() { // TODO Auto-generated method stub LOG.info("Follower Thread starts!"); while (state) { + StormClusterState zkClusterState = data.getStormClusterState(); try { Thread.sleep(sleepTime); - if (data.isLeader()) { - data.getStormClusterState() - .unregister_nimbus_host(hostPort); - checkOwnMaster(); + if (!zkClusterState.leader_existed()) { + this.tryToBeLeader(data.getConf()); continue; } - if (!data.getStormClusterState().leader_existed()) { - this.tryToBeLeader(data.getConf()); + + String master = zkClusterState.get_leader_host(); + boolean isZkLeader = isLeader(master); + if (data.isLeader() == true ) { + if (isZkLeader == false) { + LOG.info("New ZK master is " + master); + JStormUtils.halt_process(1, "Lose ZK master node, halt process"); + return ; + } + } + + if (isZkLeader == true) { + zkClusterState.unregister_nimbus_host(hostPort); + data.setLeader(true); continue; } + check(); - data.getStormClusterState().update_follower_hb(hostPort, - data.uptime()); + zkClusterState.update_follower_hb(hostPort, data.uptime()); } catch (InterruptedException e) { // TODO Auto-generated catch block continue; @@ -206,8 +231,8 @@ public void run() { } } }; - data.setLeader(data.getStormClusterState().try_to_be_leader( - Cluster.MASTER_SUBTREE, hostPort, masterCallback)); + data.getStormClusterState().try_to_be_leader( + Cluster.MASTER_SUBTREE, hostPort, masterCallback); } /** diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/MonitorRunnable.java b/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/MonitorRunnable.java index 239c4df93..6fbec61fc 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/MonitorRunnable.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/MonitorRunnable.java @@ -12,6 +12,7 @@ import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; import com.alibaba.jstorm.task.Assignment; import com.alibaba.jstorm.utils.TimeFormat; +import com.alibaba.jstorm.task.TaskInfo; /** * diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/ResourceWorkerSlot.java b/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/ResourceWorkerSlot.java index 3ed7b922e..6865bfa14 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/ResourceWorkerSlot.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/ResourceWorkerSlot.java @@ -16,6 +16,7 @@ import backtype.storm.scheduler.WorkerSlot; import com.alibaba.jstorm.client.WorkerAssignment; +import com.alibaba.jstorm.utils.NetWorkUtils; //one worker 's assignment public class ResourceWorkerSlot extends WorkerSlot implements Serializable { @@ -117,7 +118,7 @@ public boolean compareToUserDefineWorker(WorkerAssignment worker, if (jvm != null && !jvm.equals(this.jvm)) return false; String hostName = worker.getHostName(); - if (hostName != null && !hostName.equals(this.hostname)) + if (NetWorkUtils.equals(hostname, hostName) == false) return false; int port = worker.getPort(); if (port != 0 && port != this.getPort()) @@ -132,13 +133,7 @@ public boolean compareToUserDefineWorker(WorkerAssignment worker, } myComponentToNum.put(component, ++i); } - if (componentToNum.size() != myComponentToNum.size()) - return false; - for (Entry entry : componentToNum.entrySet()) { - if (myComponentToNum.get(entry.getKey()) == null - || myComponentToNum.get(entry.getKey()) != entry.getValue()) - return false; - } - return true; + + return myComponentToNum.equals(componentToNum); } } diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/WorkerMaker.java b/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/WorkerMaker.java index 79ba26d75..03a044bd8 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/WorkerMaker.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/WorkerMaker.java @@ -16,11 +16,12 @@ import backtype.storm.Config; +import com.alibaba.jstorm.client.ConfigExtension; import com.alibaba.jstorm.client.WorkerAssignment; import com.alibaba.jstorm.daemon.supervisor.SupervisorInfo; import com.alibaba.jstorm.task.Assignment; import com.alibaba.jstorm.utils.FailedAssignTopologyException; -import com.alibaba.jstorm.utils.JStromServerConfigExtension; +import com.alibaba.jstorm.utils.NetWorkUtils; public class WorkerMaker { @@ -54,11 +55,10 @@ public List makeWorkers( needAssign, result, workersNum, - getUserDefineWorkers(context, JStromServerConfigExtension + getUserDefineWorkers(context, ConfigExtension .getUserDefineAssignment(context.getStormConf()))); // old assignments - if (JStromServerConfigExtension.isUseOldAssignment(context - .getStormConf())) { + if (ConfigExtension.isUseOldAssignment(context.getStormConf())) { this.getRightWorkers(context, needAssign, result, workersNum, context.getOldWorkers()); } @@ -70,19 +70,22 @@ public List makeWorkers( for (int i = 0; i < defaultWorkerNum; i++) { result.add(new ResourceWorkerSlot()); } - - this.putAllWorkerToSupervisor(result, - this.getIsolationSupervisors(context)); - this.putAllWorkerToSupervisor(result, - this.getCanUseSupervisors(context.getCluster())); + List isolationSupervisors = this + .getIsolationSupervisors(context); + if (isolationSupervisors.size() != 0) { + this.putAllWorkerToSupervisor(result, + this.getCanUseSupervisors(isolationSupervisors)); + } else { + this.putAllWorkerToSupervisor(result, + this.getCanUseSupervisors(context.getCluster())); + } this.setAllWorkerMemAndCpu(context.getStormConf(), result); return result; } private void setAllWorkerMemAndCpu(Map conf, List result) { - long defaultSize = JStromServerConfigExtension - .getMemSizePerWorker(conf); - int defaultCpu = JStromServerConfigExtension.getCpuSlotPerWorker(conf); + long defaultSize = ConfigExtension.getMemSizePerWorker(conf); + int defaultCpu = ConfigExtension.getCpuSlotPerWorker(conf); for (ResourceWorkerSlot worker : result) { if (worker.getMemSize() <= 0) worker.setMemSize(defaultSize); @@ -98,7 +101,7 @@ private void putAllWorkerToSupervisor(List result, continue; if (worker.getHostname() != null) { for (SupervisorInfo supervisor : supervisors) { - if (supervisor.getHostName().equals(worker.getHostname()) + if (NetWorkUtils.equals(supervisor.getHostName(), worker.getHostname()) && supervisor.getWorkerPorts().size() > 0) { this.putWorkerToSupervisor(supervisor, worker); break; @@ -190,7 +193,16 @@ private void getRightWorkers(DefaultTopologyAssignContext context, private int getWorkersNum(DefaultTopologyAssignContext context, int workersNum) { Map supervisors = context.getCluster(); + List isolationSupervisors = this + .getIsolationSupervisors(context); int slotNum = 0; + + if (isolationSupervisors.size() != 0) { + for (SupervisorInfo superivsor : isolationSupervisors) { + slotNum = slotNum + superivsor.getWorkerPorts().size(); + } + return Math.min(slotNum, workersNum); + } for (Entry entry : supervisors.entrySet()) { slotNum = slotNum + entry.getValue().getWorkerPorts().size(); } @@ -268,12 +280,21 @@ private List getIsolationSupervisors( if (isolationHosts == null) return new ArrayList(); List isolationSupervisors = new ArrayList(); - for (SupervisorInfo supervisor : this.getCanUseSupervisors(context - .getCluster())) { - if (isolationHosts.contains(supervisor.getHostName())) { - isolationSupervisors.add(supervisor); + for (Entry entry : context.getCluster() + .entrySet()) { + if (containTargetHost(isolationHosts, entry.getValue().getHostName())) { + isolationSupervisors.add(entry.getValue()); } } return isolationSupervisors; } + + private boolean containTargetHost(Collection hosts, String target) { + for (String host : hosts) { + if (NetWorkUtils.equals(host, target) == true) { + return true; + } + } + return false; + } } diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/Task.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/Task.java index d774fb86c..f36cb28d9 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/Task.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/task/Task.java @@ -105,9 +105,11 @@ public Task(WorkerData workerData, int taskId) throws Exception { this.stormConf = Common.component_conf(workerData.getStormConf(), topologyContext, componentid); + WorkerClassLoader.switchThreadContext(); // get real task object -- spout/bolt/spoutspec this.taskObj = Common.get_task_object(topologyContext.getRawTopology(), componentid, WorkerClassLoader.getInstance()); + WorkerClassLoader.restoreThreadContext(); int samplerate = StormConfig.sampling_rate(stormConf); this.taskStats = new CommonStatsRolling(samplerate); diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/TaskInfo.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/TaskInfo.java index f410b3a7b..533fb1d4f 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/TaskInfo.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/task/TaskInfo.java @@ -9,7 +9,7 @@ * /storm-zk-root/tasks/{topologyid}/{taskid} data */ public class TaskInfo implements Serializable { - + private static final long serialVersionUID = 5625165079055837777L; private String componentId; private String componentType; diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/TaskMetricInfo.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/TaskMetricInfo.java index ed9a79b65..a90ff46f1 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/TaskMetricInfo.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/task/TaskMetricInfo.java @@ -6,6 +6,8 @@ import java.util.List; import java.util.ArrayList; +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; import org.apache.log4j.Logger; import com.codahale.metrics.Metric; @@ -141,4 +143,10 @@ public List anyQueueFull() { } return ret; } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } } diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/BaseExecutors.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/BaseExecutors.java index 678c44b61..4690c45f7 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/BaseExecutors.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/BaseExecutors.java @@ -5,6 +5,7 @@ import org.apache.log4j.Logger; import backtype.storm.Config; +import backtype.storm.Constants; import backtype.storm.serialization.KryoTupleDeserializer; import backtype.storm.task.TopologyContext; import backtype.storm.tuple.Tuple; @@ -15,6 +16,8 @@ import com.alibaba.jstorm.callback.AsyncLoopThread; import com.alibaba.jstorm.callback.RunnableCallback; import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.daemon.worker.timer.RotatingMapTrigger; +import com.alibaba.jstorm.daemon.worker.timer.TickTupleTrigger; import com.alibaba.jstorm.metric.JStormTimer; import com.alibaba.jstorm.metric.MetricDef; import com.alibaba.jstorm.metric.Metrics; @@ -118,6 +121,10 @@ public BaseExecutors(TaskTransfer _transfer_fn, Map _storm_conf, deserializeTimer = Metrics.registerTimer(idStr, MetricDef.DESERIALIZE_TIME, String.valueOf(taskId), Metrics.MetricType.TASK); Metrics.registerQueue(idStr, MetricDef.DESERIALIZE_QUEUE, deserializeQueue, String.valueOf(taskId), Metrics.MetricType.TASK); Metrics.registerQueue(idStr, MetricDef.EXECUTE_QUEUE, exeQueue, String.valueOf(taskId), Metrics.MetricType.TASK); + + RotatingMapTrigger rotatingMapTrigger = new RotatingMapTrigger(storm_conf, idStr + "_rotating", exeQueue); + rotatingMapTrigger.register(); + } @Override diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/BoltCollector.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/BoltCollector.java index 356ee1b98..d2e61dfc2 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/BoltCollector.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/BoltCollector.java @@ -5,8 +5,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Random; import java.util.Map.Entry; +import java.util.Random; import org.apache.log4j.Logger; @@ -17,8 +17,8 @@ import backtype.storm.tuple.Tuple; import backtype.storm.tuple.TupleImplExt; -import com.alibaba.jstorm.metric.MetricDef; import com.alibaba.jstorm.metric.JStormTimer; +import com.alibaba.jstorm.metric.MetricDef; import com.alibaba.jstorm.metric.Metrics; import com.alibaba.jstorm.stats.CommonStatsRolling; import com.alibaba.jstorm.task.TaskTransfer; @@ -30,7 +30,6 @@ import com.alibaba.jstorm.utils.JStormUtils; import com.alibaba.jstorm.utils.RotatingMap; import com.alibaba.jstorm.utils.TimeUtils; -import com.codahale.metrics.Timer; /** * bolt output interface, do emit/ack/fail diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/BoltExecutors.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/BoltExecutors.java index 1e7285496..a316c6072 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/BoltExecutors.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/BoltExecutors.java @@ -6,6 +6,7 @@ import org.apache.log4j.Logger; import backtype.storm.Config; +import backtype.storm.Constants; import backtype.storm.task.IBolt; import backtype.storm.task.IOutputCollector; import backtype.storm.task.OutputCollector; @@ -14,9 +15,10 @@ import backtype.storm.utils.DisruptorQueue; import backtype.storm.utils.WorkerClassLoader; -import com.alibaba.jstorm.daemon.worker.TimeTick; -import com.alibaba.jstorm.metric.MetricDef; +import com.alibaba.jstorm.daemon.worker.timer.RotatingMapTrigger; +import com.alibaba.jstorm.daemon.worker.timer.TickTupleTrigger; import com.alibaba.jstorm.metric.JStormTimer; +import com.alibaba.jstorm.metric.MetricDef; import com.alibaba.jstorm.metric.Metrics; import com.alibaba.jstorm.stats.CommonStatsRolling; import com.alibaba.jstorm.task.TaskStatus; @@ -90,12 +92,24 @@ public BoltExecutors(IBolt _bolt, TaskTransfer _transfer_fn, boltExeTimer = Metrics.registerTimer(idStr, MetricDef.EXECUTE_TIME, String.valueOf(taskId), Metrics.MetricType.TASK); - TimeTick.registerTimer(idStr + "-sampling-tick", exeQueue); + + Object tickFrequence = storm_conf.get(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS); + if (tickFrequence != null) { + Integer frequence = JStormUtils.parseInt(tickFrequence); + TickTupleTrigger tickTupleTrigger = new TickTupleTrigger( + sysTopologyCxt, frequence, + idStr + Constants.SYSTEM_TICK_STREAM_ID, exeQueue); + tickTupleTrigger.register(); + } try { // do prepare WorkerClassLoader.switchThreadContext(); - bolt.prepare(storm_conf, userTopologyCxt, outputCollector); + +// Method method = IBolt.class.getMethod("prepare", new Class[] {Map.class, TopologyContext.class, +// OutputCollector.class}); +// method.invoke(bolt, new Object[] {storm_conf, userTopologyCxt, outputCollector}); + bolt.prepare(storm_conf, userTopologyCtx, outputCollector); } catch (Throwable e) { error = e; @@ -145,7 +159,7 @@ public void onEvent(Object event, long sequence, boolean endOfBatch) try { - if (event instanceof TimeTick.Tick) { + if (event instanceof RotatingMapTrigger.Tick) { // don't check the timetick name to improve performance Map timeoutMap = tuple_start_times.rotate(); diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/MultipleThreadSpoutExecutors.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/MultipleThreadSpoutExecutors.java index 2298da689..2835c699b 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/MultipleThreadSpoutExecutors.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/MultipleThreadSpoutExecutors.java @@ -4,19 +4,13 @@ import org.apache.log4j.Logger; -import backtype.storm.Config; -import backtype.storm.spout.ISpoutOutputCollector; -import backtype.storm.spout.SpoutOutputCollector; import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.Tuple; import backtype.storm.utils.DisruptorQueue; import backtype.storm.utils.WorkerClassLoader; import com.alibaba.jstorm.callback.AsyncLoopThread; import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.client.ConfigExtension; import com.alibaba.jstorm.metric.MetricDef; -import com.alibaba.jstorm.metric.JStormTimer; import com.alibaba.jstorm.metric.Metrics; import com.alibaba.jstorm.stats.CommonStatsRolling; import com.alibaba.jstorm.task.TaskStatus; @@ -25,9 +19,7 @@ import com.alibaba.jstorm.task.comm.TaskSendTargets; import com.alibaba.jstorm.task.comm.TupleInfo; import com.alibaba.jstorm.task.error.ITaskReportErr; -import com.alibaba.jstorm.utils.JStormUtils; import com.alibaba.jstorm.utils.RotatingMap; -import com.alibaba.jstorm.utils.TimeCacheMap; import com.codahale.metrics.Gauge; /** diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/SpoutExecutors.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/SpoutExecutors.java index 3e72e63c6..854f43e3d 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/SpoutExecutors.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/SpoutExecutors.java @@ -14,9 +14,9 @@ import com.alibaba.jstorm.callback.AsyncLoopThread; import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.daemon.worker.TimeTick; -import com.alibaba.jstorm.metric.MetricDef; +import com.alibaba.jstorm.daemon.worker.timer.RotatingMapTrigger; import com.alibaba.jstorm.metric.JStormTimer; +import com.alibaba.jstorm.metric.MetricDef; import com.alibaba.jstorm.metric.Metrics; import com.alibaba.jstorm.stats.CommonStatsRolling; import com.alibaba.jstorm.task.TaskStatus; @@ -82,12 +82,11 @@ public SpoutExecutors(backtype.storm.spout.ISpout _spout, Metrics.register(idStr, MetricDef.EMPTY_CPU_RATIO, emptyCpuCounter, String.valueOf(taskId), Metrics.MetricType.TASK); - TimeTick.registerTimer(idStr+ "-acker-tick", exeQueue); - isSpoutFullSleep = ConfigExtension.isSpoutPendFullSleep(storm_conf); LOG.info("isSpoutFullSleep:" + isSpoutFullSleep); } + public void prepare(TaskSendTargets sendTargets, TaskTransfer transferFn, TopologyContext topologyContext) { @@ -116,14 +115,21 @@ public void prepare(TaskSendTargets sendTargets, TaskTransfer transferFn, public void nextTuple() { if (firstTime == true) { + int delayRun = ConfigExtension.getSpoutDelayRunSeconds(storm_conf); // wait other bolt is ready JStormUtils.sleepMs(delayRun * 1000); + + emptyCpuCounter.init(); + + if (taskStatus.isRun() == true) { + spout.activate(); + }else { + spout.deactivate(); + } firstTime = false; - - emptyCpuCounter.init(); LOG.info(idStr + " is ready "); } @@ -212,7 +218,7 @@ public void onEvent(Object event, long sequence, boolean endOfBatch) task_stats.recv_tuple(tuple.getSourceComponent(), tuple.getSourceStreamId()); - } else if (event instanceof TimeTick.Tick) { + } else if (event instanceof RotatingMapTrigger.Tick) { Map timeoutMap = pending.rotate(); for (java.util.Map.Entry entry : timeoutMap diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkGrouper.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkGrouper.java index 419e30d1e..15905c4a3 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkGrouper.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkGrouper.java @@ -42,6 +42,7 @@ public class MkGrouper { // grouping method private RandomRange randomrange; private Random random; + private MkShuffer shuffer; private MkCustomGrouper custom_grouper; private MkFieldsGrouper fields_grouper; private MkLocalShuffer local_shuffer_grouper; @@ -100,8 +101,8 @@ private GrouperType parseGroupType(WorkerData workerData) { // send to every task grouperType = GrouperType.all; } else if (Grouping._Fields.SHUFFLE.equals(fields)) { - this.randomrange = new RandomRange(out_tasks.size()); grouperType = GrouperType.shuffle; + shuffer = new MkShuffer(out_tasks, workerData); } else if (Grouping._Fields.NONE.equals(fields)) { // random send one task this.random = new Random(); @@ -132,11 +133,9 @@ private GrouperType parseGroupType(WorkerData workerData) { grouperType = GrouperType.direct; } else if (Grouping._Fields.LOCAL_OR_SHUFFLE.equals(fields)) { grouperType = GrouperType.local_or_shuffle; - local_shuffer_grouper = new MkLocalShuffer(local_tasks, out_tasks); - + local_shuffer_grouper = new MkLocalShuffer(local_tasks, out_tasks, workerData); }else if (Grouping._Fields.LOCAL_FIRST.equals(fields)) { - grouperType = GrouperType.localFirst; - + grouperType = GrouperType.localFirst; localFirst = new MkLocalFirst(local_tasks, out_tasks, workerData); } @@ -162,8 +161,7 @@ public List grouper(List values) { return out_tasks; } else if (GrouperType.shuffle.equals(grouptype)) { // random, but the random is different from none - int rnd = randomrange.nextInt(); - return JStormUtils.mk_list(out_tasks.get(rnd)); + return shuffer.grouper(values); } else if (GrouperType.none.equals(grouptype)) { int rnd = Math.abs(random.nextInt() % out_tasks.size()); return JStormUtils.mk_list(out_tasks.get(rnd)); diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkLocalFirst.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkLocalFirst.java index 15585dd5d..b0a9d11b0 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkLocalFirst.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkLocalFirst.java @@ -10,7 +10,6 @@ import com.alibaba.jstorm.utils.IntervalCheck; import com.alibaba.jstorm.utils.JStormUtils; import com.alibaba.jstorm.utils.RandomRange; -import com.esotericsoftware.minlog.Log; /** * @@ -18,7 +17,7 @@ * @author zhongyan.feng * @version */ -public class MkLocalFirst { +public class MkLocalFirst extends Shuffer { private static final Logger LOG = Logger.getLogger(MkLocalFirst.class); private List allOutTasks; @@ -31,6 +30,7 @@ public class MkLocalFirst { public MkLocalFirst(List workerTasks, List allOutTasks, WorkerData workerData) { + super(workerData); intervalCheck = new IntervalCheck(); intervalCheck.setInterval(60); @@ -99,10 +99,14 @@ private void refreshLocalNodeTasks() { randomrange = new RandomRange(outTasks.size()); } - private List innerGroup(List values) { - int index = randomrange.nextInt(); + private List innerGroup(List values) { + int index = getActiveTask(randomrange, outTasks); + // If none active tasks were found, still send message to a task + if (index == -1) + index = randomrange.nextInt(); + + return JStormUtils.mk_list(outTasks.get(index)); - return JStormUtils.mk_list(outTasks.get(index)); } public List grouper(List values) { diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkLocalShuffer.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkLocalShuffer.java index 1bbdf5af5..f8d36ad35 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkLocalShuffer.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkLocalShuffer.java @@ -3,16 +3,18 @@ import java.util.ArrayList; import java.util.List; +import com.alibaba.jstorm.daemon.worker.WorkerData; import com.alibaba.jstorm.utils.JStormUtils; import com.alibaba.jstorm.utils.RandomRange; -public class MkLocalShuffer { +public class MkLocalShuffer extends Shuffer{ private List outTasks; private RandomRange randomrange; private boolean isLocal; - public MkLocalShuffer(List workerTasks, List allOutTasks) { + public MkLocalShuffer(List workerTasks, List allOutTasks, WorkerData workerData) { + super(workerData); List localOutTasks = new ArrayList(); for (Integer outTask : allOutTasks) { @@ -34,7 +36,10 @@ public MkLocalShuffer(List workerTasks, List allOutTasks) { } public List grouper(List values) { - int index = randomrange.nextInt(); + int index = getActiveTask(randomrange, outTasks); + // If none active tasks were found, still send message to a task + if (index == -1) + index = randomrange.nextInt(); return JStormUtils.mk_list(outTasks.get(index)); } diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkShuffer.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkShuffer.java new file mode 100644 index 000000000..f1806fc1f --- /dev/null +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkShuffer.java @@ -0,0 +1,38 @@ +package com.alibaba.jstorm.task.group; + +import java.util.List; + +import com.alibaba.jstorm.daemon.worker.WorkerData; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.RandomRange; + +public class MkShuffer extends Shuffer{ + + private List outTasks; + private RandomRange randomrange; + + public MkShuffer(List allOutTasks, WorkerData workerData) { + super(workerData); + + outTasks = allOutTasks; + randomrange = new RandomRange(outTasks.size()); + } + + public List grouper(List values) { + int index = getActiveTask(randomrange, outTasks); + // If none active tasks were found, still send message to a task + if (index == -1) + index = randomrange.nextInt(); + + return JStormUtils.mk_list(outTasks.get(index)); + } + + /** + * @param args + */ + public static void main(String[] args) { + // TODO Auto-generated method stub + + } + +} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/Shuffer.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/Shuffer.java new file mode 100644 index 000000000..1b22d1ab0 --- /dev/null +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/Shuffer.java @@ -0,0 +1,31 @@ +package com.alibaba.jstorm.task.group; + +import java.util.List; + +import com.alibaba.jstorm.daemon.worker.WorkerData; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.RandomRange; + +public abstract class Shuffer { + private WorkerData workerData; + + public Shuffer(WorkerData workerData) { + this.workerData = workerData; + } + + public abstract List grouper(List values); + + protected int getActiveTask(RandomRange randomrange, List outTasks) { + int index = randomrange.nextInt(); + int size = outTasks.size(); + + for(int i = 0; i < size; i++) { + if(workerData.isOutboundTaskActive(Integer.valueOf(outTasks.get(index)))) + break; + else + index = randomrange.nextInt(); + } + + return (index < size ? index : -1); + } +} \ No newline at end of file diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/heartbeat/TaskHeartbeatRunable.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/heartbeat/TaskHeartbeatRunable.java index 1c7958063..f035c3e92 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/heartbeat/TaskHeartbeatRunable.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/task/heartbeat/TaskHeartbeatRunable.java @@ -18,6 +18,7 @@ import com.alibaba.jstorm.task.UptimeComputer; import com.alibaba.jstorm.utils.JStormUtils; import com.alibaba.jstorm.utils.TimeUtils; +import com.alibaba.jstorm.task.TaskInfo; /** * Task hearbeat diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/utils/JStromServerConfigExtension.java b/jstorm-server/src/main/java/com/alibaba/jstorm/utils/JStromServerConfigExtension.java deleted file mode 100644 index 6e358bef8..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/utils/JStromServerConfigExtension.java +++ /dev/null @@ -1,38 +0,0 @@ -package com.alibaba.jstorm.utils; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -import backtype.storm.utils.Utils; - -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.client.WorkerAssignment; - -public class JStromServerConfigExtension extends ConfigExtension { - - public static List getUserDefineAssignment(Map conf) { - List ret = new ArrayList(); - if (conf.get(USE_USERDEFINE_ASSIGNMENT) == null) - return ret; - for (String worker : (List) conf.get(USE_USERDEFINE_ASSIGNMENT)) { - ret.add(WorkerAssignment.parseFromObj(Utils.from_json(worker))); - } - return ret; - } - - public static boolean isUseOldAssignment(Map conf) { - return JStormUtils.parseBoolean(conf.get(USE_OLD_ASSIGNMENT), false); - } - - public static long getMemSizePerWorker(Map conf) { - long size = JStormUtils.parseLong(conf.get(MEMSIZE_PER_WORKER), - JStormUtils.SIZE_1_G * 2); - return size > 0 ? size : JStormUtils.SIZE_1_G * 2; - } - - public static int getCpuSlotPerWorker(Map conf) { - int slot = JStormUtils.parseInt(conf.get(CPU_SLOT_PER_WORKER), 1); - return slot > 0 ? slot : 1; - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/utils/Thrift.java b/jstorm-server/src/main/java/com/alibaba/jstorm/utils/Thrift.java index 75a0e174d..485b37ed0 100644 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/utils/Thrift.java +++ b/jstorm-server/src/main/java/com/alibaba/jstorm/utils/Thrift.java @@ -135,7 +135,7 @@ public static Grouping mkDirectGrouping() { return Grouping.direct(new NullStruct()); } - private static ComponentCommon mkAckerComponentcommon( + private static ComponentCommon mkComponentcommon( Map inputs, HashMap output_spec, Integer parallelism_hint) { ComponentCommon ret = new ComponentCommon(inputs, output_spec); @@ -145,9 +145,9 @@ private static ComponentCommon mkAckerComponentcommon( return ret; } - public static Bolt mkAckerBolt(Map inputs, + public static Bolt mkBolt(Map inputs, IBolt bolt, HashMap output, Integer p) { - ComponentCommon common = mkAckerComponentcommon(inputs, output, p); + ComponentCommon common = mkComponentcommon(inputs, output, p); byte[] boltSer = Utils.serialize(bolt); ComponentObject component = ComponentObject.serialized_java(boltSer); return new Bolt(component, common); diff --git a/jstorm-server/src/main/resources/defaults.yaml b/jstorm-server/src/main/resources/defaults.yaml index ecd5b66c7..29b653a05 100644 --- a/jstorm-server/src/main/resources/defaults.yaml +++ b/jstorm-server/src/main/resources/defaults.yaml @@ -15,16 +15,28 @@ storm.zookeeper.connection.timeout: 15000 storm.zookeeper.retry.times: 20 storm.zookeeper.retry.interval: 1000 storm.zookeeper.retry.intervalceiling.millis: 30000 +storm.zookeeper.auth.user: null +storm.zookeeper.auth.password: null storm.cluster.mode: "distributed" # can be distributed or local storm.local.mode.zmq: false storm.thrift.transport: "backtype.storm.security.auth.SimpleTransportPlugin" +storm.principal.tolocal: "backtype.storm.security.auth.DefaultPrincipalToLocal" +storm.group.mapping.service: "backtype.storm.security.auth.ShellBasedGroupsMapping" #storm.messaging.transport: "com.alibaba.jstorm.message.zeroMq.MQContext" storm.messaging.transport: "com.alibaba.jstorm.message.netty.NettyContext" +storm.nimbus.retry.times: 5 +storm.nimbus.retry.interval.millis: 2000 +storm.nimbus.retry.intervalceiling.millis: 60000 +storm.auth.simple-white-list.users: [] +storm.auth.simple-acl.users: [] +storm.auth.simple-acl.users.commands: [] +storm.auth.simple-acl.admins: [] +storm.meta.serialization.delegate: "backtype.storm.serialization.DefaultSerializationDelegate" ### nimbus.* configs are for the master nimbus.host: "localhost" nimbus.thrift.port: 7627 -nimbus.thrift.max_buffer_size: 1048576 +nimbus.thrift.max_buffer_size: 6291456 nimbus.childopts: " -Xms2g -Xmx2g -Xmn768m -XX:PermSize=128m -XX:SurvivorRatio=4 -XX:MaxTenuringThreshold=20 -XX:+UseConcMarkSweepGC -XX:+UseCMSInitiatingOccupancyOnly -XX:CMSInitiatingOccupancyFraction=70 -XX:CMSFullGCsBeforeCompaction=5 -XX:+HeapDumpOnOutOfMemoryError -XX:+UseCMSCompactAtFullCollection -XX:CMSMaxAbortablePrecleanTime=5000 " nimbus.task.timeout.secs: 120 nimbus.supervisor.timeout.secs: 60 @@ -36,7 +48,8 @@ nimbus.reassign: true nimbus.file.copy.expiration.secs: 600 nimbus.topology.validator: "backtype.storm.nimbus.DefaultTopologyValidator" nimbus.classpath: "" -nimbus.use.ip: false +nimbus.use.ip: true +nimbus.credential.renewers.freq.secs: 600 ### ui.* configs are for the master ui.port: 8080 @@ -44,10 +57,19 @@ ui.childopts: " -Xms1g -Xmx1g -Xmn256m -XX:PermSize=96m -XX:+UseConcMarkSweepGC drpc.port: 4772 drpc.worker.threads: 64 +drpc.max_buffer_size: 1048576 drpc.queue.size: 128 drpc.invocations.port: 4773 +drpc.invocations.threads: 64 drpc.request.timeout.secs: 600 drpc.childopts: " -Xms1g -Xmx1g -Xmn256m -XX:PermSize=96m -Xmn128m -XX:PermSize=64m -XX:+UseConcMarkSweepGC -XX:+UseCMSInitiatingOccupancyOnly -XX:CMSInitiatingOccupancyFraction=70 -XX:CMSFullGCsBeforeCompaction=5 -XX:+HeapDumpOnOutOfMemoryError -XX:+UseCMSCompactAtFullCollection -XX:CMSMaxAbortablePrecleanTime=5000 " +drpc.http.port: 3774 +drpc.https.port: -1 +drpc.https.keystore.password: "" +drpc.https.keystore.type: "JKS" +drpc.http.creds.plugin: backtype.storm.security.auth.DefaultHttpCredentialsPlugin +drpc.authorizer.acl.filename: "drpc-auth-acl.yaml" +drpc.authorizer.acl.strict: false transactional.zookeeper.root: "/transactional" transactional.zookeeper.servers: null @@ -67,12 +89,23 @@ drpc.servers: ### supervisor.* configs are for node supervisors # Define the amount of workers that can be run on this machine. Each worker is assigned a port to use for communication -supervisor.slots.ports: - - 6800 - - 6801 - - 6802 - - 6803 + +# if supervisor.slots.ports is null, +# the port list will be generated by cpu cores and system memory size +# for example, if there are 24 cpu cores and supervisor.slots.port.cpu.weight is 1.2 +# then there are 24/1.2 ports for cpu, +# there are system_physical_memory_size/worker.memory.size ports for memory +# The final port number is min(cpu_ports, memory_port) +supervisor.slots.ports.base: 6800 +supervisor.slots.port.cpu.weight: 1 +supervisor.slots.ports: null +#supervisor.slots.ports: +# - 6800 +# - 6801 +# - 6802 +# - 6803 supervisor.childopts: " -Xms512m -Xmx512m -Xmn128m -XX:PermSize=64m -XX:+UseConcMarkSweepGC -XX:+UseCMSInitiatingOccupancyOnly -XX:CMSInitiatingOccupancyFraction=70 -XX:CMSFullGCsBeforeCompaction=5 -XX:+HeapDumpOnOutOfMemoryError -XX:+UseCMSCompactAtFullCollection -XX:CMSMaxAbortablePrecleanTime=5000 " +supervisor.run.worker.as.user: false #how long supervisor will wait to ensure that a worker process is started supervisor.worker.start.timeout.secs: 120 #how long between heartbeats until supervisor considers that worker dead and tries to restart it @@ -85,13 +118,13 @@ supervisor.enable: true #if set null, it will be get by system supervisor.hostname: null # use ip -supervisor.use.ip: false +supervisor.use.ip: true ### worker.* configs are for task workers # worker gc configuration # worker.gc.path will put all gc logs and memory dump file -worker.gc.childopts: " -XX:SurvivorRatio=4 -XX:MaxTenuringThreshold=20 -XX:+UseConcMarkSweepGC -XX:+UseCMSInitiatingOccupancyOnly -XX:CMSInitiatingOccupancyFraction=70 -XX:CMSFullGCsBeforeCompaction=5 -XX:+HeapDumpOnOutOfMemoryError -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:+UseCMSCompactAtFullCollection -XX:CMSMaxAbortablePrecleanTime=5000 " +worker.gc.childopts: " -XX:SurvivorRatio=4 -XX:MaxTenuringThreshold=15 -XX:+UseConcMarkSweepGC -XX:+UseCMSInitiatingOccupancyOnly -XX:CMSInitiatingOccupancyFraction=70 -XX:CMSFullGCsBeforeCompaction=5 -XX:+HeapDumpOnOutOfMemoryError -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:+UseCMSCompactAtFullCollection -XX:CMSMaxAbortablePrecleanTime=5000 " worker.heartbeat.frequency.secs: 2 worker.classpath: "" worker.redirect.output: true @@ -145,6 +178,7 @@ topology.acker.executors: null topology.tasks: null # maximum amount of time a message has to complete before it's considered failed topology.message.timeout.secs: 30 +topology.multilang.serializer: "backtype.storm.multilang.JsonSerializer" topology.skip.missing.kryo.registrations: false topology.max.task.parallelism: null # topology.spout.parallelism and topology.bolt.parallelism are used @@ -168,7 +202,7 @@ topology.executor.receive.buffer.size: 256 #batched topology.executor.send.buffer.size: 256 #individual messages topology.receiver.buffer.size: 8 # setting it too high causes a lot of problems (heartbeat thread gets starved, throughput plummets) topology.transfer.buffer.size: 1024 # batched -topology.buffer.size.limited: false #topology queue capacity is unlimited +topology.buffer.size.limited: true #topology queue capacity is unlimited topology.tick.tuple.freq.secs: null topology.worker.shared.thread.pool.size: 4 topology.disruptor.wait.strategy: "com.lmax.disruptor.BlockingWaitStrategy" @@ -187,7 +221,7 @@ topology.alimonitor.topo.metrics.name: "jstorm_metric" topology.alimonitor.task.metrics.name: "jstorm_task_metrics" topology.alimonitor.worker.metrics.name: "jstorm_worker_metrics" topology.alimonitor.user.metrics.name: "jstorm_user_metrics" - +topology.task.error.report.interval: 60 # enable topology use user-define classloader to avoid class conflict topology.enable.classloader: false diff --git a/jstorm-server/src/test/java/com/alibaba/jstorm/message/zmq/ZmqUnitTest.java b/jstorm-server/src/test/java/com/alibaba/jstorm/message/zmq/ZmqUnitTest.java index 09d8641b8..98a378d04 100644 --- a/jstorm-server/src/test/java/com/alibaba/jstorm/message/zmq/ZmqUnitTest.java +++ b/jstorm-server/src/test/java/com/alibaba/jstorm/message/zmq/ZmqUnitTest.java @@ -42,7 +42,7 @@ public class ZmqUnitTest { // WaitStrategy waitStrategy = (WaitStrategy) Utils // .newInstance((String) storm_conf // .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); -// DisruptorQueue recvQueue = new DisruptorQueue( +// DisruptorQueue recvQueue = DisruptorQueue.mkInstance( // new SingleThreadedClaimStrategy(1024), waitStrategy); // server.registerQueue(recvQueue); // @@ -92,7 +92,7 @@ public class ZmqUnitTest { // WaitStrategy waitStrategy = (WaitStrategy) Utils // .newInstance((String) storm_conf // .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); -// DisruptorQueue recvQueue = new DisruptorQueue( +// DisruptorQueue recvQueue = DisruptorQueue.mkInstance( // new SingleThreadedClaimStrategy(1024), waitStrategy); // server.registerQueue(recvQueue); // @@ -127,7 +127,7 @@ public class ZmqUnitTest { // WaitStrategy waitStrategy = (WaitStrategy) Utils // .newInstance((String) storm_conf // .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); -// DisruptorQueue recvQueue = new DisruptorQueue( +// DisruptorQueue recvQueue = DisruptorQueue.mkInstance( // new SingleThreadedClaimStrategy(1024), waitStrategy); // server.registerQueue(recvQueue); // @@ -160,7 +160,7 @@ public class ZmqUnitTest { // WaitStrategy waitStrategy = (WaitStrategy) Utils // .newInstance((String) storm_conf // .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); -// DisruptorQueue recvQueue = new DisruptorQueue( +// DisruptorQueue recvQueue = DisruptorQueue.mkInstance( // new SingleThreadedClaimStrategy(1024), waitStrategy); // server.registerQueue(recvQueue); // @@ -217,7 +217,7 @@ public class ZmqUnitTest { // WaitStrategy waitStrategy = (WaitStrategy) Utils // .newInstance((String) storm_conf // .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); -// DisruptorQueue recvQueue = new DisruptorQueue( +// DisruptorQueue recvQueue = DisruptorQueue.mkInstance( // new SingleThreadedClaimStrategy(1024), waitStrategy); // server.registerQueue(recvQueue); // @@ -262,7 +262,7 @@ public class ZmqUnitTest { // WaitStrategy waitStrategy = (WaitStrategy) Utils // .newInstance((String) storm_conf // .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); -// DisruptorQueue recvQueue = new DisruptorQueue( +// DisruptorQueue recvQueue = DisruptorQueue.mkInstance( // new SingleThreadedClaimStrategy(1024), waitStrategy); // server.registerQueue(recvQueue); // diff --git a/jstorm-server/src/test/java/com/alibaba/jstorm/superivosr/deamon/HttpserverTest.java b/jstorm-server/src/test/java/com/alibaba/jstorm/superivosr/deamon/HttpserverTest.java index a7c1a4c53..5647bb260 100644 --- a/jstorm-server/src/test/java/com/alibaba/jstorm/superivosr/deamon/HttpserverTest.java +++ b/jstorm-server/src/test/java/com/alibaba/jstorm/superivosr/deamon/HttpserverTest.java @@ -16,26 +16,10 @@ public void test_httpserver() { Httpserver httpserver = new Httpserver(port, new HashMap()); httpserver.start(); System.out.println("start...."); - } - - enum ViewMode { - LAST_4K, LAST_8K, ALL; -// final String mode; -// -// ViewMode(String mode) { -// this.mode = mode; -// } -// -// String getMode() { -// return mode; -// } + httpserver.shutdown(); } - @Test - public void test() { -// System.out.println(ViewMode.LAST_4K.getMode()); - System.out.println(Enum.valueOf(ViewMode.class, "LAST_4K")); - } + } diff --git a/jstorm-server/src/test/java/com/alibaba/jstorm/topology/SingleJoinBolt.java b/jstorm-server/src/test/java/com/alibaba/jstorm/topology/SingleJoinBolt.java new file mode 100644 index 000000000..0c95b7999 --- /dev/null +++ b/jstorm-server/src/test/java/com/alibaba/jstorm/topology/SingleJoinBolt.java @@ -0,0 +1,128 @@ +/** + * 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.topology; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import backtype.storm.Config; +import backtype.storm.generated.GlobalStreamId; +import backtype.storm.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseRichBolt; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.utils.TimeCacheMap; + +public class SingleJoinBolt extends BaseRichBolt { + OutputCollector _collector; + Fields _idFields; + Fields _outFields; + int _numSources; + TimeCacheMap, Map> _pending; + Map _fieldLocations; + + public SingleJoinBolt(Fields outFields) { + _outFields = outFields; + } + + @Override + public void prepare(Map conf, TopologyContext context, + OutputCollector collector) { + _fieldLocations = new HashMap(); + _collector = collector; + int timeout = ((Number) conf.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS)) + .intValue(); + _pending = new TimeCacheMap, Map>( + timeout, new ExpireCallback()); + _numSources = context.getThisSources().size(); + Set idFields = null; + for (GlobalStreamId source : context.getThisSources().keySet()) { + Fields fields = context.getComponentOutputFields( + source.get_componentId(), source.get_streamId()); + Set setFields = new HashSet(fields.toList()); + if (idFields == null) + idFields = setFields; + else + idFields.retainAll(setFields); + + for (String outfield : _outFields) { + for (String sourcefield : fields) { + if (outfield.equals(sourcefield)) { + _fieldLocations.put(outfield, source); + } + } + } + } + _idFields = new Fields(new ArrayList(idFields)); + + if (_fieldLocations.size() != _outFields.size()) { + throw new RuntimeException( + "Cannot find all outfields among sources"); + } + } + + @Override + public void execute(Tuple tuple) { + List id = tuple.select(_idFields); + GlobalStreamId streamId = new GlobalStreamId( + tuple.getSourceComponent(), tuple.getSourceStreamId()); + if (!_pending.containsKey(id)) { + _pending.put(id, new HashMap()); + } + Map parts = _pending.get(id); + if (parts.containsKey(streamId)) + throw new RuntimeException( + "Received same side of single join twice"); + parts.put(streamId, tuple); + if (parts.size() == _numSources) { + _pending.remove(id); + List joinResult = new ArrayList(); + for (String outField : _outFields) { + GlobalStreamId loc = _fieldLocations.get(outField); + joinResult.add(parts.get(loc).getValueByField(outField)); + } + _collector.emit(new ArrayList(parts.values()), joinResult); + + for (Tuple part : parts.values()) { + _collector.ack(part); + } + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(_outFields); + } + + private class ExpireCallback + implements + TimeCacheMap.ExpiredCallback, Map> { + @Override + public void expire(List id, Map tuples) { + for (Tuple tuple : tuples.values()) { + _collector.fail(tuple); + } + } + } +} diff --git a/jstorm-server/src/test/java/com/alibaba/jstorm/topology/SingleJoinTest.java b/jstorm-server/src/test/java/com/alibaba/jstorm/topology/SingleJoinTest.java new file mode 100644 index 000000000..f85a7bbcc --- /dev/null +++ b/jstorm-server/src/test/java/com/alibaba/jstorm/topology/SingleJoinTest.java @@ -0,0 +1,79 @@ +/** + * 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.topology; + +import junit.framework.Assert; + +import org.apache.log4j.Logger; +import org.junit.Test; + +import backtype.storm.Config; +import backtype.storm.LocalCluster; +import backtype.storm.testing.FeederSpout; +import backtype.storm.topology.TopologyBuilder; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Values; + +import com.alibaba.jstorm.utils.JStormUtils; + +public class SingleJoinTest { + private static Logger LOG = Logger.getLogger(SingleJoinTest.class); + + @Test + public void test_single_join() { + try { + FeederSpout genderSpout = new FeederSpout( + new Fields("id", "gender")); + FeederSpout ageSpout = new FeederSpout(new Fields("id", "age")); + + TopologyBuilder builder = new TopologyBuilder(); + builder.setSpout("gender", genderSpout); + builder.setSpout("age", ageSpout); + builder.setBolt("join", + new SingleJoinBolt(new Fields("gender", "age"))) + .fieldsGrouping("gender", new Fields("id")) + .fieldsGrouping("age", new Fields("id")); + + Config conf = new Config(); + conf.setDebug(true); + + LocalCluster cluster = new LocalCluster(); + cluster.submitTopology("join-example", conf, + builder.createTopology()); + + for (int i = 0; i < 10; i++) { + String gender; + if (i % 2 == 0) { + gender = "male"; + } else { + gender = "female"; + } + genderSpout.feed(new Values(i, gender)); + } + + for (int i = 9; i >= 0; i--) { + ageSpout.feed(new Values(i, i + 20)); + } + + JStormUtils.sleepMs(60 * 1000); + cluster.shutdown(); + } catch (Exception e) { + Assert.fail("Failed to run SingleJoinExample"); + } + } +} diff --git a/jstorm-server/src/test/java/com/alibaba/jstorm/topology/TransactionalWordsTest.java b/jstorm-server/src/test/java/com/alibaba/jstorm/topology/TransactionalWordsTest.java new file mode 100644 index 000000000..75524cd5e --- /dev/null +++ b/jstorm-server/src/test/java/com/alibaba/jstorm/topology/TransactionalWordsTest.java @@ -0,0 +1,274 @@ +/** + * 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.topology; + +import backtype.storm.Config; +import backtype.storm.LocalCluster; +import backtype.storm.coordination.BatchOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.testing.MemoryTransactionalSpout; +import backtype.storm.topology.BasicOutputCollector; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseBasicBolt; +import backtype.storm.topology.base.BaseTransactionalBolt; +import backtype.storm.transactional.ICommitter; +import backtype.storm.transactional.TransactionAttempt; +import backtype.storm.transactional.TransactionalTopologyBuilder; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; + +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import junit.framework.Assert; + +import org.junit.Test; + +import com.alibaba.jstorm.utils.JStormUtils; + +/** + * This class defines a more involved transactional topology then + * TransactionalGlobalCount. This topology processes a stream of words and + * produces two outputs: + *

+ * 1. A count for each word (stored in a database) 2. The number of words for + * every bucket of 10 counts. So it stores in the database how many words have + * appeared 0-9 times, how many have appeared 10-19 times, and so on. + *

+ * A batch of words can cause the bucket counts to decrement for some buckets + * and increment for others as words move between buckets as their counts + * accumulate. + */ +public class TransactionalWordsTest { + public static class CountValue { + Integer prev_count = null; + int count = 0; + BigInteger txid = null; + } + + public static class BucketValue { + int count = 0; + BigInteger txid; + } + + public static final int BUCKET_SIZE = 10; + + public static Map COUNT_DATABASE = new HashMap(); + public static Map BUCKET_DATABASE = new HashMap(); + + public static final int PARTITION_TAKE_PER_BATCH = 3; + + public static final Map>> DATA = new HashMap>>() { + { + put(0, new ArrayList>() { + { + add(new Values("cat")); + add(new Values("dog")); + add(new Values("chicken")); + add(new Values("cat")); + add(new Values("dog")); + add(new Values("apple")); + } + }); + put(1, new ArrayList>() { + { + add(new Values("cat")); + add(new Values("dog")); + add(new Values("apple")); + add(new Values("banana")); + } + }); + put(2, new ArrayList>() { + { + add(new Values("cat")); + add(new Values("cat")); + add(new Values("cat")); + add(new Values("cat")); + add(new Values("cat")); + add(new Values("dog")); + add(new Values("dog")); + add(new Values("dog")); + add(new Values("dog")); + } + }); + } + }; + + public static class KeyedCountUpdater extends BaseTransactionalBolt + implements ICommitter { + Map _counts = new HashMap(); + BatchOutputCollector _collector; + TransactionAttempt _id; + + int _count = 0; + + @Override + public void prepare(Map conf, TopologyContext context, + BatchOutputCollector collector, TransactionAttempt id) { + _collector = collector; + _id = id; + } + + @Override + public void execute(Tuple tuple) { + String key = tuple.getString(1); + Integer curr = _counts.get(key); + if (curr == null) + curr = 0; + _counts.put(key, curr + 1); + } + + @Override + public void finishBatch() { + for (String key : _counts.keySet()) { + CountValue val = COUNT_DATABASE.get(key); + CountValue newVal; + if (val == null || !val.txid.equals(_id.getTransactionId())) { + newVal = new CountValue(); + newVal.txid = _id.getTransactionId(); + if (val != null) { + newVal.prev_count = val.count; + newVal.count = val.count; + } + newVal.count = newVal.count + _counts.get(key); + COUNT_DATABASE.put(key, newVal); + } else { + newVal = val; + } + _collector.emit(new Values(_id, key, newVal.count, + newVal.prev_count)); + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("id", "key", "count", "prev-count")); + } + } + + public static class Bucketize extends BaseBasicBolt { + @Override + public void execute(Tuple tuple, BasicOutputCollector collector) { + TransactionAttempt attempt = (TransactionAttempt) tuple.getValue(0); + int curr = tuple.getInteger(2); + Integer prev = tuple.getInteger(3); + + int currBucket = curr / BUCKET_SIZE; + Integer prevBucket = null; + if (prev != null) { + prevBucket = prev / BUCKET_SIZE; + } + + if (prevBucket == null) { + collector.emit(new Values(attempt, currBucket, 1)); + } else if (currBucket != prevBucket) { + collector.emit(new Values(attempt, currBucket, 1)); + collector.emit(new Values(attempt, prevBucket, -1)); + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("attempt", "bucket", "delta")); + } + } + + public static class BucketCountUpdater extends BaseTransactionalBolt { + Map _accum = new HashMap(); + BatchOutputCollector _collector; + TransactionAttempt _attempt; + + int _count = 0; + + @Override + public void prepare(Map conf, TopologyContext context, + BatchOutputCollector collector, TransactionAttempt attempt) { + _collector = collector; + _attempt = attempt; + } + + @Override + public void execute(Tuple tuple) { + Integer bucket = tuple.getInteger(1); + Integer delta = tuple.getInteger(2); + Integer curr = _accum.get(bucket); + if (curr == null) + curr = 0; + _accum.put(bucket, curr + delta); + } + + @Override + public void finishBatch() { + for (Integer bucket : _accum.keySet()) { + BucketValue currVal = BUCKET_DATABASE.get(bucket); + BucketValue newVal; + if (currVal == null + || !currVal.txid.equals(_attempt.getTransactionId())) { + newVal = new BucketValue(); + newVal.txid = _attempt.getTransactionId(); + newVal.count = _accum.get(bucket); + if (currVal != null) + newVal.count += currVal.count; + BUCKET_DATABASE.put(bucket, newVal); + } else { + newVal = currVal; + } + _collector.emit(new Values(_attempt, bucket, newVal.count)); + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("id", "bucket", "count")); + } + } + + @Test + public void test_transaction_word(){ + try { + MemoryTransactionalSpout spout = new MemoryTransactionalSpout(DATA, + new Fields("word"), PARTITION_TAKE_PER_BATCH); + TransactionalTopologyBuilder builder = new TransactionalTopologyBuilder( + "top-n-words", "spout", spout, 2); + builder.setBolt("count", new KeyedCountUpdater(), 5).fieldsGrouping( + "spout", new Fields("word")); + builder.setBolt("bucketize", new Bucketize()).shuffleGrouping("count"); + builder.setBolt("buckets", new BucketCountUpdater(), 5).fieldsGrouping( + "bucketize", new Fields("bucket")); + + LocalCluster cluster = new LocalCluster(); + + Config config = new Config(); + config.setDebug(true); + config.setMaxSpoutPending(3); + + cluster.submitTopology("top-n-topology", config, + builder.buildTopology()); + + JStormUtils.sleepMs(60 * 1000); + cluster.shutdown(); + }catch(Exception e) { + Assert.fail("Failed to run simple transaction"); + } + + } +} diff --git a/jstorm-ui/pom.xml b/jstorm-ui/pom.xml index 613aa2dc1..1c4a00429 100644 --- a/jstorm-ui/pom.xml +++ b/jstorm-ui/pom.xml @@ -4,7 +4,7 @@ com.alibaba.jstorm jstorm-all - 0.9.6.2 + 0.9.6.3 .. diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/UIUtils.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/UIUtils.java index 95ffed191..11615376d 100644 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/UIUtils.java +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/UIUtils.java @@ -467,11 +467,14 @@ public static List clusterSummary(ClusterSummary summ, ClusterSumm clusterSumm = new ClusterSumm(); String master = client.getMasterHost(); - clusterSumm.setNimbusHostname(master); + if (master.contains(":")) { - clusterSumm.setNimbusIp(NetWorkUtils.host2Ip(master.substring(0, - master.indexOf(":")))); + String firstPart = master.substring(0, master.indexOf(":") ); + String lastPart = master.substring(master.indexOf(":")); + clusterSumm.setNimbusHostname(NetWorkUtils.ip2Host(firstPart) + lastPart); + clusterSumm.setNimbusIp(NetWorkUtils.host2Ip(firstPart)); } else { + clusterSumm.setNimbusHostname(master); clusterSumm.setNimbusIp(NetWorkUtils.host2Ip(master)); } int port = ConfigExtension.getNimbusDeamonHttpserverPort(conf); diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/NimbusSlave.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/NimbusSlave.java index 7286b9cf4..596fc7dac 100644 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/NimbusSlave.java +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/NimbusSlave.java @@ -19,10 +19,18 @@ public class NimbusSlave implements Serializable { private String uptime; public NimbusSlave(String hostname, String uptime, int port) { - this.hostname = hostname; + + if (hostname.contains(":")) { + String firstPart = hostname.substring(0, hostname.indexOf(":") ); + String lastPart = hostname.substring(hostname.indexOf(":")); + this.hostname = (NetWorkUtils.ip2Host(firstPart) + lastPart); + this.ip = (NetWorkUtils.host2Ip(firstPart)); + } else { + this.hostname = hostname; + this.ip = (NetWorkUtils.host2Ip(hostname)); + } + this.uptime = uptime; - String[] fields = StringUtils.split(hostname, ":"); - this.ip = NetWorkUtils.host2Ip(fields[0]); this.logviewPort = String.valueOf(port); } diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/ConfPage.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/ConfPage.java new file mode 100644 index 000000000..bb427aa8f --- /dev/null +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/ConfPage.java @@ -0,0 +1,180 @@ +package com.alibaba.jstorm.ui.model.data; + +import java.io.File; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import javax.faces.bean.ManagedBean; +import javax.faces.bean.ViewScoped; +import javax.faces.context.FacesContext; + +import org.apache.commons.lang.StringUtils; +import org.apache.http.HttpResponse; +import org.apache.http.client.HttpClient; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.impl.client.HttpClientBuilder; +import org.apache.http.util.EntityUtils; +import org.apache.log4j.Logger; +import org.apache.thrift7.TException; + +import backtype.storm.generated.NotAliveException; +import backtype.storm.generated.TaskSummary; +import backtype.storm.generated.TopologyInfo; +import backtype.storm.utils.NimbusClient; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.ui.UIUtils; +import com.alibaba.jstorm.ui.model.ComponentTask; +import com.alibaba.jstorm.ui.model.LogPageIndex; +import com.alibaba.jstorm.utils.HttpserverUtils; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.NetWorkUtils; + +/** + * task log view page service.
+ * implement view the specified task log through proxy way. current support + * + * + * @author L + * @version 1.0.0 <2014-04-20 21:23> + * @since JDK1.6 + */ +@ManagedBean(name = "confpage") +@ViewScoped +public class ConfPage implements Serializable { + + private static final long serialVersionUID = 4326599394273506083L; + + private static final Logger LOG = Logger.getLogger(ConfPage.class); + + /** + * proxy url, which call the log service on the task node. + */ + private static final String PROXY_URL = "http://%s:%s/logview?%s=%s"; + + private String confData = ""; + + private String host; + + private int port; + + private Map conf; + + public ConfPage() throws Exception { + FacesContext ctx = FacesContext.getCurrentInstance(); + if (ctx.getExternalContext().getRequestParameterMap().get("host") != null) { + host = ctx.getExternalContext().getRequestParameterMap() + .get("host"); + } + + if (ctx.getExternalContext().getRequestParameterMap().get("port") != null) { + + port = JStormUtils.parseInt(ctx.getExternalContext() + .getRequestParameterMap().get("port"), 0); + } + + init(); + } + + private void init() throws Exception { + + + try { + + conf = UIUtils.readUiConfig(); + + if (port == 0) { + port = ConfigExtension.getSupervisorDeamonHttpserverPort(conf); + } + + // proxy call + queryConf(); + + } catch (Exception e) { + LOG.error(e.getCause(), e); + throw e; + } + } + + + /** + * proxy query log for the specified task. + * + * @param task + * the specified task + */ + private void queryConf() { + // PROXY_URL = "http://%s:%s/logview?%s=%s&log=%s"; + String baseUrl = String.format(PROXY_URL, NetWorkUtils.host2Ip(host), port, + HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_CMD, + HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_CMD_SHOW_CONF); + String url = baseUrl; + + try { + // 1. proxy call the task host log view service + HttpClient client = HttpClientBuilder.create().build(); + HttpPost post = new HttpPost(url); + HttpResponse response = client.execute(post); + + // 2. check the request is success, then read the log + if (response.getStatusLine().getStatusCode() == 200) { + String data = EntityUtils.toString(response.getEntity(), ConfigExtension.getLogViewEncoding(conf)); + + setConfData(parseJsonConf(data)); + } else { + setConfData(EntityUtils.toString(response.getEntity())); + } + } catch (Exception e) { + setConfData(e.getMessage()); + LOG.error(e.getCause(), e); + } + } + + private String parseJsonConf(String jsonData) { + Map remoteConf = (Map)Utils.from_json(jsonData); + + StringBuilder sb = new StringBuilder(); + + for (Entry entry : remoteConf.entrySet()) { + sb.append(entry.getKey()); + sb.append(":"); + sb.append(entry.getValue()); + sb.append("\n"); + } + + return sb.toString(); + } + + public String getConfData() { + return confData; + } + + public void setConfData(String confData) { + this.confData = confData; + } + + public String getHost() { + return host; + } + + public void setHost(String host) { + this.host = host; + } + + public int getPort() { + return port; + } + + public void setPort(int port) { + this.port = port; + } + + + + + +} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/JStackPage.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/JStackPage.java index 1cbc6bb90..d6650874f 100644 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/JStackPage.java +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/JStackPage.java @@ -18,6 +18,7 @@ import com.alibaba.jstorm.ui.UIUtils; import com.alibaba.jstorm.utils.HttpserverUtils; import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.NetWorkUtils; @ManagedBean(name = "jstackpage") @@ -98,7 +99,7 @@ private void init() throws Exception { private void queryLog(Map conf) { // PROXY_URL = "http://%s:%s/logview?%s=%s&%s=%s"; String baseUrl = String - .format(PROXY_URL, host, port, + .format(PROXY_URL, NetWorkUtils.host2Ip(host), port, HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_CMD, HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_CMD_JSTACK, HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_WORKER_PORT, diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/ListLogPage.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/ListLogPage.java index 9d7ff28b6..f7c370355 100644 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/ListLogPage.java +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/ListLogPage.java @@ -22,6 +22,7 @@ import com.alibaba.jstorm.utils.FileAttribute; import com.alibaba.jstorm.utils.HttpserverUtils; import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.NetWorkUtils; /** * task log view page service.
@@ -158,7 +159,7 @@ private void parseString(String input) { private void listLogs() { // PROXY_URL = "http://%s:%s/logview?%s=%s&dir=%s"; - String url = String.format(PROXY_URL, host, port, + String url = String.format(PROXY_URL, NetWorkUtils.host2Ip(host), port, HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_CMD, HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_CMD_LIST, HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_DIR, parent); diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/LogPage.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/LogPage.java index 9f760d63d..cae1a69e4 100644 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/LogPage.java +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/LogPage.java @@ -30,6 +30,7 @@ import com.alibaba.jstorm.ui.model.LogPageIndex; import com.alibaba.jstorm.utils.HttpserverUtils; import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.NetWorkUtils; /** * task log view page service.
@@ -306,7 +307,7 @@ private void genPageUrl(String sizeStr) { */ private void queryLog(Map conf) { // PROXY_URL = "http://%s:%s/logview?%s=%s&log=%s"; - String baseUrl = String.format(PROXY_URL, host, port, + String baseUrl = String.format(PROXY_URL, NetWorkUtils.host2Ip(host), port, HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_CMD, HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_CMD_SHOW, logFileName); String url = baseUrl; diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/MainPage.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/MainPage.java index ebef2a347..59d4a441a 100644 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/MainPage.java +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/MainPage.java @@ -88,10 +88,12 @@ private void init() throws Exception { } catch (Exception e) { String errorInfo = e.getMessage(); - if (errorInfo.indexOf("No alive nimbus") == -1) { - LOG.error("Failed to get cluster information:", e); - throw e; - } + if (errorInfo != null) { + if (errorInfo.indexOf("No alive nimbus") == -1) { + LOG.error("Failed to get cluster information:", e); + throw e; + } + } } finally { if (client != null) { client.close(); @@ -203,4 +205,4 @@ public static void main(String[] args) { } -} \ No newline at end of file +} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/TopologyPage.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/TopologyPage.java index d5b7fcd4e..73a197841 100644 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/TopologyPage.java +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/TopologyPage.java @@ -22,9 +22,11 @@ import backtype.storm.generated.StormTopology; import backtype.storm.generated.TaskSummary; import backtype.storm.generated.TopologyInfo; +import backtype.storm.generated.UserDefMetric; import backtype.storm.utils.NimbusClient; import com.alibaba.jstorm.common.stats.StatBuckets; +import com.alibaba.jstorm.metric.UserDefMetricData; import com.alibaba.jstorm.ui.UIUtils; import com.alibaba.jstorm.ui.model.Components; import com.alibaba.jstorm.ui.model.TopologySumm; @@ -50,6 +52,7 @@ public class TopologyPage implements Serializable { private List tstats = null; private List scom = null; private List bcom = null; + private List udm = null; public TopologyPage() throws Exception { @@ -99,6 +102,8 @@ private void init() throws Exception { StormTopology topology = client.getClient().getTopology(topologyid); List ts = summ.get_tasks(); + + udm = summ.get_userDefMetric(); tsumm = UIUtils.topologySummary(summ); @@ -220,6 +225,14 @@ private List topologyStatsTable(List scom, tss.add(topologyStats); return tss; } + + public List getUdm() { + return udm; + } + + public void setUdm(List udm) { + this.udm = udm; + } public String getTopologyid() { return topologyid; diff --git a/jstorm-ui/src/main/webapp/cluster.xhtml b/jstorm-ui/src/main/webapp/cluster.xhtml index e75c72712..8cfa636bf 100644 --- a/jstorm-ui/src/main/webapp/cluster.xhtml +++ b/jstorm-ui/src/main/webapp/cluster.xhtml @@ -66,6 +66,17 @@ + + + + + + + + + + + @@ -99,6 +110,18 @@ + + + + + + + + + + + + diff --git a/jstorm-ui/src/main/webapp/conf.xhtml b/jstorm-ui/src/main/webapp/conf.xhtml new file mode 100644 index 000000000..899ded742 --- /dev/null +++ b/jstorm-ui/src/main/webapp/conf.xhtml @@ -0,0 +1,26 @@ + + + + + + + Jstorm UI Configuration + + + +

Host: '#{confpage.host}'

+ + +
+
+    
+	
+ +
+ + + diff --git a/jstorm-ui/src/main/webapp/spout.xhtml b/jstorm-ui/src/main/webapp/spout.xhtml index a7824b5eb..31f045831 100644 --- a/jstorm-ui/src/main/webapp/spout.xhtml +++ b/jstorm-ui/src/main/webapp/spout.xhtml @@ -208,7 +208,7 @@ - + diff --git a/jstorm-ui/src/main/webapp/supervisor.xhtml b/jstorm-ui/src/main/webapp/supervisor.xhtml index a7406ba90..bbea6f725 100644 --- a/jstorm-ui/src/main/webapp/supervisor.xhtml +++ b/jstorm-ui/src/main/webapp/supervisor.xhtml @@ -55,6 +55,17 @@ + + + + + + + + + + + diff --git a/jstorm-ui/src/main/webapp/topology.xhtml b/jstorm-ui/src/main/webapp/topology.xhtml index 500fa8c2d..eacc82e9a 100644 --- a/jstorm-ui/src/main/webapp/topology.xhtml +++ b/jstorm-ui/src/main/webapp/topology.xhtml @@ -301,6 +301,34 @@ + +

User Defined Metrics

+ + + + + + + + + + + + + + + + + + + + + + + + + +